diff --git a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/ParquetDataFormatPlugin.java b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/ParquetDataFormatPlugin.java index 3c686bc37f209..94a837e3bf81a 100644 --- a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/ParquetDataFormatPlugin.java +++ b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/ParquetDataFormatPlugin.java @@ -11,12 +11,20 @@ import com.parquet.parquetdataformat.fields.ArrowSchemaBuilder; import com.parquet.parquetdataformat.engine.read.ParquetDataSourceCodec; import com.parquet.parquetdataformat.writer.ParquetWriter; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DataFormatPlugin; import org.opensearch.index.engine.exec.DataFormat; import org.opensearch.index.engine.exec.IndexingExecutionEngine; import com.parquet.parquetdataformat.bridge.RustBridge; import com.parquet.parquetdataformat.engine.ParquetExecutionEngine; import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.FormatStoreDirectory; +import org.opensearch.index.store.GenericStoreDirectory; import org.opensearch.plugins.DataSourcePlugin; import org.opensearch.index.mapper.MapperService; import org.opensearch.plugins.Plugin; @@ -26,6 +34,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.Set; /** * OpenSearch plugin that provides Parquet data format support for indexing operations. @@ -58,16 +67,17 @@ */ public class ParquetDataFormatPlugin extends Plugin implements DataFormatPlugin, DataSourcePlugin { + /** + * Set of file extensions that Parquet format handles + */ + private static final Set PARQUET_EXTENSIONS = Set.of(".parquet", ".pqt"); + @Override @SuppressWarnings("unchecked") public IndexingExecutionEngine indexingEngine(MapperService mapperService, ShardPath shardPath) { return (IndexingExecutionEngine) new ParquetExecutionEngine(() -> ArrowSchemaBuilder.getSchema(mapperService), shardPath); } - private Class getDataFormatType() { - return ParquetDataFormat.class; - } - @Override public DataFormat getDataFormat() { return new ParquetDataFormat(); @@ -83,6 +93,28 @@ public Optional createFormatStoreDirectory( + IndexSettings indexSettings, + ShardPath shardPath + ) throws IOException { + Logger logger = LogManager.getLogger("index.store.parquet." + shardPath.getShardId()); + + return new GenericStoreDirectory<>( + new ParquetDataFormat(), + shardPath.getDataPath(), + logger + ); + } + + @Override + public BlobContainer createBlobContainer(BlobStore blobStore, BlobPath baseBlobPath) throws IOException + { + BlobPath formatPath = baseBlobPath.add(getDataFormat().name().toLowerCase()); + return blobStore.blobContainer(formatPath); + } + // for testing locally only public void indexDataToParquetEngine() throws IOException { //Create Engine (take Schema as Input) diff --git a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/engine/ParquetExecutionEngine.java b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/engine/ParquetExecutionEngine.java index 3d3596f9feef9..a642a9babab5e 100644 --- a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/engine/ParquetExecutionEngine.java +++ b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/engine/ParquetExecutionEngine.java @@ -97,7 +97,7 @@ public List supportedFieldTypes() { @Override public Writer createWriter(long writerGeneration) throws IOException { - String fileName = Path.of(shardPath.getDataPath().toString(), FILE_NAME_PREFIX + "_" + writerGeneration + FILE_NAME_EXT).toString(); + String fileName = Path.of(shardPath.getDataPath().toString(), getDataFormat().name(), FILE_NAME_PREFIX + "_" + writerGeneration + FILE_NAME_EXT).toString(); return new ParquetWriter(fileName, schema.get(), writerGeneration); } diff --git a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/merge/RecordBatchMergeStrategy.java b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/merge/RecordBatchMergeStrategy.java index 36f9930ded673..bc271eb507d7a 100644 --- a/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/merge/RecordBatchMergeStrategy.java +++ b/modules/parquet-data-format/src/main/java/com/parquet/parquetdataformat/merge/RecordBatchMergeStrategy.java @@ -56,6 +56,7 @@ public MergeResult mergeParquetFiles(Collection files) { Map rowIdMapping = new HashMap<>(); FileMetadata mergedFileMetadata = new FileMetadata( + "", outputDirectory, mergedFileName ); diff --git a/plugins/engine-datafusion/jni/src/lib.rs b/plugins/engine-datafusion/jni/src/lib.rs index 788a5e8651cca..314ecaa80b611 100644 --- a/plugins/engine-datafusion/jni/src/lib.rs +++ b/plugins/engine-datafusion/jni/src/lib.rs @@ -360,7 +360,7 @@ pub extern "system" fn Java_org_opensearch_datafusion_DataFusionQueryJNI_execute let substrait_plan = match Plan::decode(plan_bytes_vec.as_slice()) { Ok(plan) => { - // println!("SUBSTRAIT rust: Decoding is successful, Plan has {} relations", plan.relations.len()); + println!("SUBSTRAIT rust: Decoding is successful, Plan has {} relations", plan.relations.len()); plan }, Err(e) => { diff --git a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/DatafusionEngine.java b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/DatafusionEngine.java index 1f32127486455..89a88195a7dbc 100644 --- a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/DatafusionEngine.java +++ b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/DatafusionEngine.java @@ -66,8 +66,8 @@ public class DatafusionEngine extends SearchExecEngine formatCatalogSnapshot, DataFusionService dataFusionService, ShardPath shardPath) throws IOException { this.dataFormat = dataFormat; - - this.datafusionReaderManager = new DatafusionReaderManager(shardPath.getDataPath().toString(), formatCatalogSnapshot, dataFormat.getName()); + String path = shardPath.getDataPath().resolve(dataFormat.getName()).toString(); + this.datafusionReaderManager = new DatafusionReaderManager(path, formatCatalogSnapshot, dataFormat.getName()); this.datafusionService = dataFusionService; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/metadata/TransportRemoteStoreMetadataAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/metadata/TransportRemoteStoreMetadataAction.java index 16c29d7586a98..c01bb198dda44 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/metadata/TransportRemoteStoreMetadataAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/metadata/TransportRemoteStoreMetadataAction.java @@ -23,6 +23,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.store.RemoteSegmentStoreDirectory; @@ -209,7 +210,7 @@ private Map> getSegmentMetadata( RemoteSegmentMetadata segmentMetadata = entry.getValue(); Map segmentMetadataMap = new HashMap<>(); - Map filesMap = new HashMap<>(); + Map filesMap = new HashMap<>(); segmentMetadata.getMetadata().forEach((file, meta) -> { Map metaMap = new HashMap<>(); metaMap.put("original_name", meta.getOriginalFilename()); 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..8845c12099079 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 = "1.0.0") public interface BlobContainer { /** @@ -276,6 +278,7 @@ default void writeBlobAtomicWithMetadata( /** * The type representing sort order of blob names + * @opensearch.api */ enum BlobNameSortOrder { 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..9d29df318efee 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java @@ -32,6 +32,8 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.PublicApi; + /** * An interface for providing basic metadata about a blob. * 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..e25f026bcd4a8 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 + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class DeleteResult { public static final DeleteResult ZERO = new DeleteResult(0, 0); diff --git a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java index cbd1852202d1c..5289c2a652a27 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java @@ -34,6 +34,8 @@ import java.util.zip.CRC32; import com.jcraft.jzlib.JZlib; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.indices.replication.common.ReplicationLuceneIndex; /** * RemoteTransferContainer is an encapsulation for managing file transfers. @@ -57,11 +59,12 @@ public class RemoteTransferContainer implements Closeable { private final boolean isRemoteDataIntegritySupported; private final AtomicBoolean readBlock = new AtomicBoolean(); private final Map metadata; + private final String dataFormat; private static final Logger log = LogManager.getLogger(RemoteTransferContainer.class); /** - * Construct a new RemoteTransferContainer object + * OLD Construct a new RemoteTransferContainer object * * @param fileName Name of the local file * @param remoteFileName Name of the remote file @@ -91,10 +94,35 @@ public RemoteTransferContainer( offsetRangeInputStreamSupplier, expectedChecksum, isRemoteDataIntegritySupported, + null, null ); } + public RemoteTransferContainer( + FileMetadata fileMetadata, + String remoteFileName, + long contentLength, + boolean failTransferIfFileExists, + WritePriority writePriority, + OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier, + Long expectedChecksum, + boolean isRemoteDataIntegritySupported + ) { + this( + fileMetadata.file(), + remoteFileName, + contentLength, + failTransferIfFileExists, + writePriority, + offsetRangeInputStreamSupplier, + expectedChecksum, + isRemoteDataIntegritySupported, + null, + fileMetadata.dataFormat() + ); + } + /** * Construct a new RemoteTransferContainer object with metadata. * @@ -117,7 +145,8 @@ public RemoteTransferContainer( OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier, Long expectedChecksum, boolean isRemoteDataIntegritySupported, - Map metadata + Map metadata, + String dataFormat ) { this.fileName = fileName; this.remoteFileName = remoteFileName; @@ -128,6 +157,7 @@ public RemoteTransferContainer( this.expectedChecksum = expectedChecksum; this.isRemoteDataIntegritySupported = isRemoteDataIntegritySupported; this.metadata = metadata; + this.dataFormat = dataFormat; } /** diff --git a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java index b62ae1f1d3956..acb537c764ca1 100644 --- a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java +++ b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java @@ -61,7 +61,9 @@ public VersionedCodecStreamWrapper( public T readStream(IndexInput indexInput) throws IOException { logger.debug("Reading input stream [{}] of length - [{}]", indexInput.toString(), indexInput.length()); try { - CodecUtil.checksumEntireFile(indexInput); + // Todo: @kamal currently we are not adding checksum to the footer so getting this error + // Caused by: org.apache.lucene.index.CorruptIndexException: codec footer mismatch (file truncated?): actual footer=2 vs expected footer=-1071082520 (resource=BufferedChecksumIndexInput(metadata file)) + // CodecUtil.checksumEntireFile(indexInput); int readStreamVersion = checkHeader(indexInput); return getHandlerForVersion(readStreamVersion).readContent(indexInput); } catch (CorruptIndexException cie) { @@ -84,7 +86,8 @@ public T readStream(IndexInput indexInput) throws IOException { public void writeStream(IndexOutput indexOutput, T content) throws IOException { this.writeHeader(indexOutput); getHandlerForVersion(this.currentVersion).writeContent(indexOutput, content); - this.writeFooter(indexOutput); + // Todo: @Kamal, This API doesn't work with Parquet files. + // this.writeFooter(indexOutput); } /** diff --git a/server/src/main/java/org/opensearch/common/util/UploadListener.java b/server/src/main/java/org/opensearch/common/util/UploadListener.java index 66feb68f7bf0d..41e642f43542d 100644 --- a/server/src/main/java/org/opensearch/common/util/UploadListener.java +++ b/server/src/main/java/org/opensearch/common/util/UploadListener.java @@ -8,6 +8,8 @@ package org.opensearch.common.util; +import org.opensearch.index.engine.exec.FileMetadata; + /** * A tracker class that is fed to FileUploader. * @@ -15,9 +17,9 @@ */ public interface UploadListener { - void beforeUpload(String file); + void beforeUpload(FileMetadata fileMetadata); - void onSuccess(String file); + void onSuccess(FileMetadata fileMetadata); - void onFailure(String file); + void onFailure(FileMetadata fileMetadata); } diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index f715dd13cd25f..25a3a6c6317de 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -77,6 +77,7 @@ import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.index.similarity.SimilarityService; +import org.opensearch.index.store.CompositeStoreDirectoryFactory; import org.opensearch.index.store.DefaultCompositeDirectoryFactory; import org.opensearch.index.store.FsDirectoryFactory; import org.opensearch.index.store.Store; @@ -267,6 +268,7 @@ public final class IndexModule { private final Map> similarities = new HashMap<>(); private final Map directoryFactories; private final Map compositeDirectoryFactories; + private final Map compositeStoreDirectoryFactories; private final SetOnce> forceQueryCacheProvider = new SetOnce<>(); private final List searchOperationListeners = new ArrayList<>(); private final List indexOperationListeners = new ArrayList<>(); @@ -300,7 +302,8 @@ public IndexModule( final Map recoveryStateFactories, final Map storeFactories, final FileCache fileCache, - final CompositeIndexSettings compositeIndexSettings + final CompositeIndexSettings compositeIndexSettings, + final Map compositeStoreDirectoryFactories ) { this.indexSettings = indexSettings; this.analysisRegistry = analysisRegistry; @@ -310,6 +313,7 @@ public IndexModule( this.indexOperationListeners.add(new IndexingSlowLog(indexSettings)); this.directoryFactories = Collections.unmodifiableMap(directoryFactories); this.compositeDirectoryFactories = Collections.unmodifiableMap(compositeDirectoryFactories); + this.compositeStoreDirectoryFactories = Collections.unmodifiableMap(compositeStoreDirectoryFactories); this.allowExpensiveQueries = allowExpensiveQueries; this.expressionResolver = expressionResolver; this.recoveryStateFactories = recoveryStateFactories; @@ -341,7 +345,8 @@ public IndexModule( recoveryStateFactories, Collections.emptyMap(), null, - null + null, + Collections.emptyMap() ); } @@ -762,6 +767,10 @@ public IndexService newIndexService( indexSettings, compositeDirectoryFactories ); + final CompositeStoreDirectoryFactory compositeStoreDirectoryFactory = getCompositeStoreDirectoryFactory( + indexSettings, + compositeStoreDirectoryFactories + ); final IndexStorePlugin.RecoveryStateFactory recoveryStateFactory = getRecoveryStateFactory(indexSettings, recoveryStateFactories); QueryCache queryCache = null; IndexAnalyzers indexAnalyzers = null; @@ -825,7 +834,8 @@ public IndexService newIndexService( segmentReplicationStatsProvider, clusterDefaultMaxMergeAtOnceSupplier, searchEnginePlugin, - pluginsService + pluginsService, + compositeStoreDirectoryFactory ); success = true; return indexService; @@ -884,6 +894,18 @@ private static IndexStorePlugin.CompositeDirectoryFactory getCompositeDirectoryF return factory; } + private static CompositeStoreDirectoryFactory getCompositeStoreDirectoryFactory( + final IndexSettings indexSettings, + final Map compositeStoreDirectoryFactories + ) { + if (compositeStoreDirectoryFactories.isEmpty()) { + return null; + } + // For now, return the default factory if available + // In future, could add index setting to select specific factory type + return compositeStoreDirectoryFactories.get("default"); + } + private static IndexStorePlugin.RecoveryStateFactory getRecoveryStateFactory( final IndexSettings indexSettings, final Map recoveryStateFactories diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 277daf2696b17..e04e70f0eb86a 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -94,6 +94,8 @@ import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; +import org.opensearch.index.store.CompositeStoreDirectory; +import org.opensearch.index.store.CompositeStoreDirectoryFactory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.store.remote.filecache.FileCache; @@ -158,6 +160,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final ShardStoreDeleter shardStoreDeleter; private final IndexStorePlugin.DirectoryFactory directoryFactory; private final IndexStorePlugin.CompositeDirectoryFactory compositeDirectoryFactory; + private final CompositeStoreDirectoryFactory compositeStoreDirectoryFactory; private final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory; private final IndexStorePlugin.RecoveryStateFactory recoveryStateFactory; private final CheckedFunction readerWrapper; @@ -258,7 +261,8 @@ public IndexService( Function segmentReplicationStatsProvider, Supplier clusterDefaultMaxMergeAtOnceSupplier, SearchEnginePlugin searchEnginePlugin, - PluginsService pluginsService + PluginsService pluginsService, + CompositeStoreDirectoryFactory compositeStoreDirectoryFactory ) { super(indexSettings); this.storeFactory = storeFactory; @@ -327,6 +331,7 @@ public IndexService( this.nodeEnv = nodeEnv; this.directoryFactory = directoryFactory; this.compositeDirectoryFactory = compositeDirectoryFactory; + this.compositeStoreDirectoryFactory = compositeStoreDirectoryFactory; this.remoteDirectoryFactory = remoteDirectoryFactory; this.recoveryStateFactory = recoveryStateFactory; this.engineFactory = Objects.requireNonNull(engineFactory); @@ -457,7 +462,8 @@ public IndexService( (shardId) -> ReplicationStats.empty(), clusterDefaultMaxMergeAtOnce, searchEnginePlugin, - pluginsService + pluginsService, + null ); } @@ -730,7 +736,7 @@ protected void closeInternal() { // Do nothing for shard lock on remote store } }; - remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, remoteStoreLock, Store.OnClose.EMPTY, path); + remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, remoteStoreLock, Store.OnClose.EMPTY, path, pluginsService); } else { // Disallow shards with remote store based settings to be created on non-remote store enabled nodes // Even though we have `RemoteStoreMigrationAllocationDecider` in place to prevent something like this from happening at the @@ -744,7 +750,7 @@ protected void closeInternal() { } } - Directory directory = null; + Directory directory; if (FeatureFlags.isEnabled(FeatureFlags.WRITABLE_WARM_INDEX_SETTING) && // TODO : Need to remove this check after support for hot indices is added in Composite Directory this.indexSettings.isWarmIndex()) { @@ -759,13 +765,18 @@ protected void closeInternal() { } else { directory = directoryFactory.newDirectory(this.indexSettings, path); } - store = storeFactory.newStore( + + CompositeStoreDirectory compositeStoreDirectory = createCompositeStoreDirectory(path); + + store = new Store( shardId, this.indexSettings, directory, lock, new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)), - path + path, + pluginsService, + compositeStoreDirectory ); eventListener.onStoreCreated(shardId); indexShard = new IndexShard( @@ -811,6 +822,12 @@ protected void closeInternal() { ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); + + // ToDO:@Kamal, Discuss a better place for it's initialization, + // Complete CompositeEngine initialization after remote store stats trackers have been created + // This ensures that RemoteStoreRefreshListener gets a non-null segmentTracker + indexShard.completeCompositeEngineInitialization(); + shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); success = true; return indexShard; @@ -1345,6 +1362,24 @@ final IndexStorePlugin.DirectoryFactory getDirectoryFactory() { return directoryFactory; } // pkg private for testing + /** + * Creates CompositeStoreDirectory using the factory if available, otherwise fallback to Store's internal creation. + * This method centralizes the directory creation logic and enables plugin-based format discovery. + */ + private CompositeStoreDirectory createCompositeStoreDirectory(ShardPath shardPath) throws IOException { + if (compositeStoreDirectoryFactory != null) { + logger.debug("Using CompositeStoreDirectoryFactory to create directory for shard path: {}", shardPath); + return compositeStoreDirectoryFactory.newCompositeStoreDirectory( + indexSettings, + shardPath, + pluginsService + ); + } + + logger.debug("No CompositeStoreDirectoryFactory available, Store will handle internal creation for: {}", shardPath); + return null; + } + private void maybeFSyncTranslogs() { if (indexSettings.getTranslogDurability() == Translog.Durability.ASYNC) { for (IndexShard shard : this.shards.values()) { diff --git a/server/src/main/java/org/opensearch/index/engine/exec/FileMetadata.java b/server/src/main/java/org/opensearch/index/engine/exec/FileMetadata.java index 3878eb156654b..1f129a45d08ea 100644 --- a/server/src/main/java/org/opensearch/index/engine/exec/FileMetadata.java +++ b/server/src/main/java/org/opensearch/index/engine/exec/FileMetadata.java @@ -8,9 +8,9 @@ package org.opensearch.index.engine.exec; -public record FileMetadata(String directory, String file) { - +public record FileMetadata(String dataFormat, String directory, String file) { + @Override public String toString() { - return "FileMetadata {" + "directory='" + directory + '\'' + ", file='" + file + '\'' + '}'; + return "FileMetadata {" + "directory='" + directory + '\'' + ", file='" + file + '\'' + ", format='" + dataFormat + '\'' + '}'; } } diff --git a/server/src/main/java/org/opensearch/index/engine/exec/commit/LuceneCommitEngine.java b/server/src/main/java/org/opensearch/index/engine/exec/commit/LuceneCommitEngine.java index 7d05314d03b3c..2fb8567b376ac 100644 --- a/server/src/main/java/org/opensearch/index/engine/exec/commit/LuceneCommitEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/exec/commit/LuceneCommitEngine.java @@ -13,6 +13,7 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.store.NIOFSDirectory; import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.RefreshResult; import org.opensearch.index.engine.exec.WriterFileSet; import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.seqno.SequenceNumbers; @@ -54,6 +55,9 @@ public void addLuceneIndexes(CatalogSnapshot catalogSnapshot) { @Override public CommitPoint commit(Iterable> commitData, CatalogSnapshot catalogSnapshot) { + if(catalogSnapshot == null) { + catalogSnapshot = new CatalogSnapshot(new RefreshResult(), 0, 0); + } addLuceneIndexes(catalogSnapshot); indexWriter.setLiveCommitData(commitData); try { diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshot.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshot.java index baf3f540b53ab..a8cbf03ee5042 100644 --- a/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshot.java +++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshot.java @@ -19,8 +19,17 @@ import org.opensearch.index.engine.exec.RefreshResult; import org.opensearch.index.engine.exec.WriterFileSet; +import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.io.Serializable; +import java.io.DataOutputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectOutputStream; +import java.io.ObjectInputStream; +import java.io.InputStream; +import java.io.DataInputStream; +import java.io.ByteArrayInputStream; import java.nio.file.Path; import java.util.ArrayList; import java.util.Base64; @@ -36,13 +45,21 @@ public class CatalogSnapshot extends AbstractRefCounted implements Writeable { public static final String CATALOG_SNAPSHOT_KEY = "_catalog_snapshot_"; public static final String LAST_COMPOSITE_WRITER_GEN_KEY = "_last_composite_writer_gen_"; private final long id; + private long version; + private Map userData; private long lastWriterGeneration; private final Map> dfGroupedSearchableFiles; - public CatalogSnapshot(RefreshResult refreshResult, long id) { + // Todo: @Kamal, update version increment logic properly + public CatalogSnapshot(RefreshResult refreshResult, long id, long version) { super("catalog_snapshot"); this.id = id; + this.version = version; + this.userData = new HashMap<>(); this.dfGroupedSearchableFiles = new HashMap<>(); + if (refreshResult != null) { + refreshResult.getRefreshedFiles().forEach((dataFormat, writerFiles) -> dfGroupedSearchableFiles.put(dataFormat.name(), writerFiles)); + } this.lastWriterGeneration = -1; refreshResult.getRefreshedFiles().forEach((dataFormat, writerFiles) -> { dfGroupedSearchableFiles.put(dataFormat.name(), writerFiles); @@ -76,16 +93,20 @@ public CatalogSnapshot(StreamInput in) throws IOException { public CatalogSnapshot remapPaths(Path newShardDataPath) { Map> remappedFiles = new HashMap<>(); + for (Map.Entry> entry : dfGroupedSearchableFiles.entrySet()) { String dataFormat = entry.getKey(); List remappedFileSets = new ArrayList<>(); + Path shardDataPath = newShardDataPath.resolve(dataFormat); for (WriterFileSet fileSet : entry.getValue()) { // Create new WriterFileSet with updated directory and file paths - WriterFileSet remappedFileSet = fileSet.withDirectory(newShardDataPath.toString()); + WriterFileSet remappedFileSet = fileSet.withDirectory(shardDataPath.toString()); remappedFileSets.add(remappedFileSet); } + remappedFiles.put(dataFormat, remappedFileSets); } + return new CatalogSnapshot(this.id, remappedFiles); } @@ -133,6 +154,66 @@ public Collection getSegments() { return Collections.unmodifiableCollection(segmentMap.values()); } + public Collection getFileMetadataList() { + Collection segments = getSegments(); + Collection allFileMetadata = new ArrayList<>(); + + for (Segment segment : segments) { + segment.dfGroupedSearchableFiles.forEach((dataFormatName, writerFileSet) -> { + for (String filePath : writerFileSet.getFiles()) { + File file = new File(filePath); + String fileName = file.getName(); + FileMetadata fileMetadata = new FileMetadata( + dataFormatName, + writerFileSet.getDirectory(), + fileName + ); + allFileMetadata.add(fileMetadata); + } + }); + } + + return allFileMetadata; + } + + public long getGeneration() { + return id; + } + + public long getVersion() { + return version; + } + + /** + * Returns user data associated with this catalog snapshot. + * + * @return map of user data key-value pairs + */ + public Map getUserData() { + return new HashMap<>(userData); + } + + /** + * Sets user data for this catalog snapshot. + * + * @param userData the user data map to set + * @param doIncrementVersion whether to increment version (for compatibility with SegmentInfos API) + */ + public void setUserData(Map userData, boolean doIncrementVersion) { + if (userData == null) { + this.userData = Collections.emptyMap(); + } else { + this.userData = new HashMap<>(userData); + } + if (doIncrementVersion) { + changed(); + } + } + + public void changed() { + version++; + } + @Override protected void closeInternal() { // notify to file deleter, search, etc @@ -142,6 +223,136 @@ public long getId() { return id; } + /** + * Writes this CatalogSnapshot to an OutputStream using standard Java serialization. + * This method serializes the catalog snapshot data without relying on Lucene-specific APIs. + * + * @param outputStream the OutputStream to write to + * @throws IOException if an I/O error occurs during writing + */ + public void writeTo(OutputStream outputStream) throws IOException { + DataOutputStream dataOut = new DataOutputStream(outputStream); + + // Write basic metadata + dataOut.writeLong(id); + dataOut.writeLong(version); + + // Write userData map + dataOut.writeInt(userData.size()); + for (Map.Entry entry : userData.entrySet()) { + dataOut.writeUTF(entry.getKey()); + dataOut.writeUTF(entry.getValue()); + } + + // Write dfGroupedSearchableFiles map + dataOut.writeInt(dfGroupedSearchableFiles.size()); + for (Map.Entry> entry : dfGroupedSearchableFiles.entrySet()) { + dataOut.writeUTF(entry.getKey()); // data format name + + Collection writerFileSets = entry.getValue(); + dataOut.writeInt(writerFileSets.size()); + + // Serialize each WriterFileSet using ObjectOutputStream + for (WriterFileSet writerFileSet : writerFileSets) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(writerFileSet); + oos.flush(); + + byte[] serializedWriterFileSet = baos.toByteArray(); + dataOut.writeInt(serializedWriterFileSet.length); + dataOut.write(serializedWriterFileSet); + } + } + + dataOut.flush(); + } + + /** + * Reads a CatalogSnapshot from an InputStream using standard Java deserialization. + * This method deserializes catalog snapshot data without relying on Lucene-specific APIs. + * + * @param inputStream the InputStream to read from + * @return a new CatalogSnapshot instance + * @throws IOException if an I/O error occurs during reading + * @throws ClassNotFoundException if a class cannot be found during deserialization + */ + public static CatalogSnapshot readFrom(InputStream inputStream) throws IOException, ClassNotFoundException { + DataInputStream dataIn = new DataInputStream(inputStream); + + // Read basic metadata + long id = dataIn.readLong(); + long version = dataIn.readLong(); + + // Create CatalogSnapshot instance + CatalogSnapshot catalog = new CatalogSnapshot(null, id, version); + + // Read userData map + int userDataSize = dataIn.readInt(); + Map userData = new HashMap<>(); + for (int i = 0; i < userDataSize; i++) { + String key = dataIn.readUTF(); + String value = dataIn.readUTF(); + userData.put(key, value); + } + catalog.userData = userData; + + // Read dfGroupedSearchableFiles map + int dfGroupedSize = dataIn.readInt(); + for (int i = 0; i < dfGroupedSize; i++) { + String dataFormat = dataIn.readUTF(); + + int writerFileSetsSize = dataIn.readInt(); + Collection writerFileSets = new ArrayList<>(); + + for (int j = 0; j < writerFileSetsSize; j++) { + int serializedSize = dataIn.readInt(); + byte[] serializedData = new byte[serializedSize]; + dataIn.readFully(serializedData); + + ByteArrayInputStream bais = new ByteArrayInputStream(serializedData); + ObjectInputStream ois = new ObjectInputStream(bais); + WriterFileSet writerFileSet = (WriterFileSet) ois.readObject(); + writerFileSets.add(writerFileSet); + } + + catalog.dfGroupedSearchableFiles.put(dataFormat, writerFileSets); + } + + return catalog; + } + + /** + * Creates a clone of this CatalogSnapshot with the same content but independent lifecycle. + * Similar to SegmentInfos.clone(), this allows creating independent copies that can be + * modified without affecting the original snapshot. + * + * @return a new CatalogSnapshot instance with the same data + */ + public CatalogSnapshot clone() { + CatalogSnapshot cloned = new CatalogSnapshot(null, this.id, this.version); + cloned.userData = new HashMap<>(this.userData); + + for (Map.Entry> entry : this.dfGroupedSearchableFiles.entrySet()) { + String dataFormat = entry.getKey(); + Collection writerFileSets = entry.getValue(); + + Collection clonedWriterFileSets = new ArrayList<>(writerFileSets); + cloned.dfGroupedSearchableFiles.put(dataFormat, clonedWriterFileSets); + } + + return cloned; + } + + // @Override + // public String toString() { + // return "CatalogSnapshot{" + + // "id=" + id + + // ", version=" + version + + // ", dfGroupedSearchableFiles=" + dfGroupedSearchableFiles + + // '}'; + // } + public long getLastWriterGeneration() { return lastWriterGeneration; } @@ -168,8 +379,8 @@ public void addSearchableFiles(String dataFormat, WriterFileSet writerFileSetGro public Collection getSearchableFiles(String df) { List searchableFiles = new ArrayList<>(); String directory = dfGroupedSearchableFiles.get(df).getDirectory(); - for (String file : dfGroupedSearchableFiles.get(df).getFiles()) { - searchableFiles.add(new FileMetadata(directory, file)); + for(String file : dfGroupedSearchableFiles.get(df).getFiles()) { + searchableFiles.add(new FileMetadata(df ,directory, file)); } return searchableFiles; } diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngine.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngine.java index c351a51bc4198..efa3b93b8f40c 100644 --- a/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngine.java @@ -14,6 +14,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lease.Releasable; import org.opensearch.common.logging.Loggers; import org.opensearch.common.util.concurrent.KeyedLock; @@ -37,6 +38,7 @@ import org.opensearch.index.engine.SafeCommitInfo; import org.opensearch.index.engine.SearchExecEngine; import org.opensearch.index.engine.Segment; +import org.opensearch.index.engine.EngineConfig; import org.opensearch.index.engine.VersionValue; import org.opensearch.index.engine.exec.DataFormat; import org.opensearch.index.engine.exec.FileMetadata; @@ -382,6 +384,43 @@ LocalCheckpointTracker getLocalCheckpointTracker() { return localCheckpointTracker; } + public void updateSearchEngine() throws IOException { + catalogSnapshotAwareRefreshListeners.forEach(ref -> { + try { + ref.afterRefresh(true, catalogSnapshot); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + /** + * Initialize refresh listeners from EngineConfig after all dependencies are ready. + * This method should be called after remote store stats trackers have been created. + * ToDo: Added as part of upload flow test, Need to discuss. + */ + public void initializeRefreshListeners(EngineConfig engineConfig) { + // Add EngineConfig refresh listeners to catalogSnapshotAwareRefreshListeners + if (engineConfig.getInternalRefreshListener() != null) { + for (ReferenceManager.RefreshListener listener : engineConfig.getInternalRefreshListener()) { + if (listener instanceof CatalogSnapshotAwareRefreshListener) { + catalogSnapshotAwareRefreshListeners.add((CatalogSnapshotAwareRefreshListener) listener); + } + } + } + + // Also check external refresh listeners + if (engineConfig.getExternalRefreshListener() != null) { + for (ReferenceManager.RefreshListener listener : engineConfig.getExternalRefreshListener()) { + if (listener instanceof CatalogSnapshotAwareRefreshListener) { + catalogSnapshotAwareRefreshListeners.add((CatalogSnapshotAwareRefreshListener) listener); + } + } + } + + System.out.println("CompositeEngine initialized with " + catalogSnapshotAwareRefreshListeners.size() + " catalog snapshot aware refresh listeners"); + } + public SearchExecEngine getReadEngine(org.opensearch.vectorized.execution.search.DataFormat dataFormat) { return readEngines.getOrDefault(dataFormat, new ArrayList<>()).getFirst(); } @@ -671,8 +710,11 @@ public synchronized void refresh(String source, Map re refreshListeners.forEach(PRE_REFRESH_LISTENER_CONSUMER); long id = 0L; + long version = 0L; if (catalogSnapshot != null) { id = catalogSnapshot.getId(); + version = catalogSnapshot.getVersion(); + } CatalogSnapshot newCatSnap; try { @@ -685,7 +727,7 @@ public synchronized void refresh(String source, Map re return; } } - newCatSnap = new CatalogSnapshot(refreshResult, id + 1L); + newCatSnap = new CatalogSnapshot(refreshResult, id + 1L, version + 1L); System.out.println("CATALOG SNAPSHOT: " + newCatSnap); } catch (IOException ex) { throw new RuntimeException(ex); @@ -746,6 +788,22 @@ public void triggerPossibleMerges() { } } + public GatedCloseable getCatalogSnapshotReference() { + if(catalogSnapshot == null) { + return new GatedCloseable<>(null, () -> {}); + } + + catalogSnapshot.incRef(); + return new GatedCloseable<>(catalogSnapshot, () -> catalogSnapshot.decRef()); + } + + public void setCatalogSnapshot(CatalogSnapshot catalogSnapshot, ShardPath shardPath) { + this.catalogSnapshot = catalogSnapshot; + if(this.catalogSnapshot != null) { + this.catalogSnapshot = this.catalogSnapshot.remapPaths(shardPath.getDataPath()); + } + } + // This should get wired into searcher acquireSnapshot for initializing reader context later // this now becomes equivalent of the reader // Each search side specific impl can decide on how to init specific reader instances using this pit snapshot provided by writers diff --git a/server/src/main/java/org/opensearch/index/engine/exec/merge/ParquetMergeHandler.java b/server/src/main/java/org/opensearch/index/engine/exec/merge/ParquetMergeHandler.java index f2e1dce8da021..abee93e6accb8 100644 --- a/server/src/main/java/org/opensearch/index/engine/exec/merge/ParquetMergeHandler.java +++ b/server/src/main/java/org/opensearch/index/engine/exec/merge/ParquetMergeHandler.java @@ -72,7 +72,7 @@ public Collection findForceMerges(int maxSegmentCount) { Set files = new HashSet<>(); for (ParquetTieredMergePolicy.ParquetFileInfo file : mergeGroup) { Path path = Path.of(file.getSegmentName()); - files.add(new FileMetadata(path.getParent().toString(), path.getFileName().toString())); + files.add(new FileMetadata(PARQUET_DATAFORMAT, path.getParent().toString(), path.getFileName().toString())); } oneMerges.add(new OneMerge(compositeIndexingExecutionEngine.getDataFormat().getDataFormats().get(0), files)); } @@ -107,7 +107,7 @@ public Collection findMerges() { Set files = new HashSet<>(); for (ParquetTieredMergePolicy.ParquetFileInfo file : mergeGroup) { Path path = Path.of(file.getSegmentName()); - files.add(new FileMetadata(path.getParent().toString(), path.getFileName().toString())); + files.add(new FileMetadata(PARQUET_DATAFORMAT, path.getParent().toString(), path.getFileName().toString())); } oneMerges.add(new OneMerge(compositeIndexingExecutionEngine.getDataFormat().getDataFormats().get(0), files)); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java index 4114a14b455e7..2059657081f32 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java @@ -19,6 +19,7 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.store.DirectoryFileTransferTracker; import java.io.IOException; @@ -102,20 +103,39 @@ public class RemoteSegmentTransferTracker extends RemoteTransferTracker { /** * Keeps track of segment files and their size in bytes which are part of the most recent refresh. + * Uses FileMetadata for format-aware tracking. */ - private final Map latestLocalFileNameLengthMap = ConcurrentCollections.newConcurrentMap(); + private final Map latestLocalFileNameLengthMap = ConcurrentCollections.newConcurrentMap(); /** * This contains the files from the last successful remote refresh and ongoing uploads. This gets reset to just the * last successful remote refresh state on successful remote refresh. + * Uses FileMetadata for format-aware tracking. */ - private final Set latestUploadedFiles = ConcurrentCollections.newConcurrentSet(); + private final Set latestUploadedFiles = ConcurrentCollections.newConcurrentSet(); + + /** + * Tracks format-specific upload statistics for monitoring and troubleshooting. + * Maps format name to upload count for format-aware monitoring. + */ + private final Map formatUploadCounts = ConcurrentCollections.newConcurrentMap(); + + /** + * Tracks format-specific upload bytes for detailed monitoring. + * Maps format name to total bytes uploaded for that format. + */ + private final Map formatUploadBytes = ConcurrentCollections.newConcurrentMap(); /** * Keeps the bytes lag computed so that we do not compute it for every request. */ private volatile long bytesLag; + /** + * Keeps track of format-specific upload failures for better error analysis and recovery. + */ + private final Map formatFailureCountMap = ConcurrentCollections.newConcurrentMap(); + /** * Holds count of consecutive failures until last success. Gets reset to zero if there is a success. */ @@ -161,6 +181,19 @@ public void incrementTotalUploadsSucceeded() { failures.record(false); } + /** + * Gets all format failure counts for monitoring and debugging. + * + * @return a map of format names to failure counts + */ + public Map getAllFormatFailureCounts() { + return formatFailureCountMap.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().get() + )); + } + public long getLocalRefreshSeqNo() { return localRefreshSeqNo; } @@ -292,15 +325,35 @@ void incrementRejectionCount(String rejectionReason) { incrementRejectionCount(); } - long getRejectionCount(String rejectionReason) { - return rejectionCountMap.get(rejectionReason).get(); - } + /** + * Updates the latestLocalFileNameLengthMap directly from FileMetadata map. + * Uses same conditional logic as the original String-based method - only updates files not in map or with size 0. + * + * @param fileMetadataToSizeMap map of FileMetadata to their sizes + */ + public void updateLatestLocalFileNameLengthMap(Map fileMetadataToSizeMap) { + logger.debug( + "fileMetadataPostRefresh={} latestLocalFileNamesBeforeMapUpdate={}", + fileMetadataToSizeMap.keySet(), + latestLocalFileNameLengthMap.keySet() + ); + + // Update the map - SAME CONDITIONAL LOGIC as original String-based method + fileMetadataToSizeMap.entrySet().stream() + .filter(entry -> EXCLUDE_FILES.contains(entry.getKey().file()) == false) + .filter(entry -> latestLocalFileNameLengthMap.containsKey(entry.getKey()) == false || + latestLocalFileNameLengthMap.get(entry.getKey()) == 0L) + .forEach(entry -> { + latestLocalFileNameLengthMap.put(entry.getKey(), entry.getValue()); + }); - public Map getLatestLocalFileNameLengthMap() { - return Collections.unmodifiableMap(latestLocalFileNameLengthMap); + Set fileMetadataSet = new HashSet<>(fileMetadataToSizeMap.keySet()); + latestLocalFileNameLengthMap.entrySet().removeIf(entry -> fileMetadataSet.contains(entry.getKey()) == false); + + computeBytesLag(); } - /** + /** ToDo: Remove this API, Tests still uses this API * Updates the latestLocalFileNameLengthMap by adding file name and it's size to the map. * The method is given a function as an argument which is used for determining the file size (length in bytes). * This method is also provided the collection of segment files which are the latest refresh local segment files. @@ -331,35 +384,86 @@ public Map updateLatestLocalFileNameLengthMap( } catch (IOException e) { logger.warn(new ParameterizedMessage("Exception while reading the fileLength of file={}", file), e); } - latestLocalFileNameLengthMap.put(file, fileSize); + FileMetadata fileMetadata = new FileMetadata("lucene", "", file); + latestLocalFileNameLengthMap.put(fileMetadata, fileSize); }); Set fileSet = new HashSet<>(segmentFiles); // Remove keys from the fileSizeMap that do not exist in the latest segment files latestLocalFileNameLengthMap.entrySet().removeIf(entry -> fileSet.contains(entry.getKey()) == false); computeBytesLag(); - return Collections.unmodifiableMap(latestLocalFileNameLengthMap); + return null; } + /** + * Adds a file to latestUploadedFiles using FileMetadata. + * @param fileMetadata the file metadata to add + */ + public void addToLatestUploadedFiles(FileMetadata fileMetadata) { + this.latestUploadedFiles.add(fileMetadata); + computeBytesLag(); + } + + /** + * ToDo: @Remove this API, currently used in Tests. + * String-based method for backward compatibility. + * Searches for matching FileMetadata with this filename and adds it. + * @param file the filename to add + */ public void addToLatestUploadedFiles(String file) { - this.latestUploadedFiles.add(file); + // Find matching FileMetadata for this filename + latestLocalFileNameLengthMap.keySet().stream() + .filter(fm -> fm.file().equals(file)) + .forEach(this.latestUploadedFiles::add); computeBytesLag(); } - public void setLatestUploadedFiles(Set files) { + /** + * Sets latestUploadedFiles using FileMetadata. + * @param fileMetadataSet the set of FileMetadata to set + */ + public void setLatestUploadedFiles(Set fileMetadataSet) { this.latestUploadedFiles.clear(); - this.latestUploadedFiles.addAll(files); + this.latestUploadedFiles.addAll(fileMetadataSet); computeBytesLag(); } + /** + * Gets all format upload statistics as an immutable map. + * @return map of format name to upload count + */ + public Map getFormatUploadCounts() { + return formatUploadCounts.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().get() + )); + } + + /** + * Gets all format upload byte statistics as an immutable map. + * @return map of format name to total bytes uploaded + */ + public Map getFormatUploadBytesMap() { + return formatUploadBytes.entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().get() + )); + } + private void computeBytesLag() { if (latestLocalFileNameLengthMap.isEmpty()) { return; } - Set filesNotYetUploaded = latestLocalFileNameLengthMap.keySet() + // Now using FileMetadata for format-aware tracking + Set filesNotYetUploaded = latestLocalFileNameLengthMap.keySet() .stream() - .filter(f -> !latestUploadedFiles.contains(f)) + .filter(fileMetadata -> !latestUploadedFiles.contains(fileMetadata)) .collect(Collectors.toSet()); - this.bytesLag = filesNotYetUploaded.stream().map(latestLocalFileNameLengthMap::get).mapToLong(Long::longValue).sum(); + this.bytesLag = filesNotYetUploaded.stream() + .map(latestLocalFileNameLengthMap::get) + .mapToLong(Long::longValue) + .sum(); } int getConsecutiveFailureCount() { @@ -392,7 +496,10 @@ public RemoteSegmentTransferTracker.Stats stats() { uploadTimeMsMovingAverageReference.get().getAverage(), getBytesLag(), totalUploadTimeInMillis.get(), - directoryFileTransferTracker.stats() + directoryFileTransferTracker.stats(), + getFormatUploadCounts(), + getFormatUploadBytesMap(), + getAllFormatFailureCounts() ); } @@ -425,6 +532,9 @@ public static class Stats implements Writeable { public final double uploadTimeMovingAverage; public final long bytesLag; public final DirectoryFileTransferTracker.Stats directoryFileTransferTrackerStats; + public final Map formatUploadCounts; + public final Map formatUploadBytes; + public final Map formatFailureCounts; public Stats( ShardId shardId, @@ -447,7 +557,10 @@ public Stats( double uploadTimeMovingAverage, long bytesLag, long totalUploadTimeInMs, - DirectoryFileTransferTracker.Stats directoryFileTransferTrackerStats + DirectoryFileTransferTracker.Stats directoryFileTransferTrackerStats, + Map formatUploadCounts, + Map formatUploadBytes, + Map formatFailureCounts ) { this.shardId = shardId; this.localRefreshClockTimeMs = localRefreshClockTimeMs; @@ -470,6 +583,9 @@ public Stats( this.bytesLag = bytesLag; this.totalUploadTimeInMs = totalUploadTimeInMs; this.directoryFileTransferTrackerStats = directoryFileTransferTrackerStats; + this.formatUploadCounts = Collections.unmodifiableMap(formatUploadCounts); + this.formatUploadBytes = Collections.unmodifiableMap(formatUploadBytes); + this.formatFailureCounts = Collections.unmodifiableMap(formatFailureCounts); } public Stats(StreamInput in) throws IOException { @@ -495,6 +611,11 @@ public Stats(StreamInput in) throws IOException { this.bytesLag = in.readLong(); this.totalUploadTimeInMs = in.readLong(); this.directoryFileTransferTrackerStats = in.readOptionalWriteable(DirectoryFileTransferTracker.Stats::new); + + // Read format-aware statistics (with backward compatibility) + this.formatUploadCounts = in.readMap(StreamInput::readString, StreamInput::readLong); + this.formatUploadBytes = in.readMap(StreamInput::readString, StreamInput::readLong); + this.formatFailureCounts = in.readMap(StreamInput::readString, StreamInput::readLong); } catch (IOException e) { throw e; } @@ -523,6 +644,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(bytesLag); out.writeLong(totalUploadTimeInMs); out.writeOptionalWriteable(directoryFileTransferTrackerStats); + + // Write format-aware statistics + out.writeMap(formatUploadCounts, StreamOutput::writeString, StreamOutput::writeLong); + out.writeMap(formatUploadBytes, StreamOutput::writeString, StreamOutput::writeLong); + out.writeMap(formatFailureCounts, StreamOutput::writeString, StreamOutput::writeLong); } @Override @@ -551,7 +677,10 @@ public boolean equals(Object obj) { && Double.compare(this.uploadTimeMovingAverage, other.uploadTimeMovingAverage) == 0 && this.bytesLag == other.bytesLag && this.totalUploadTimeInMs == other.totalUploadTimeInMs - && this.directoryFileTransferTrackerStats.equals(other.directoryFileTransferTrackerStats); + && this.directoryFileTransferTrackerStats.equals(other.directoryFileTransferTrackerStats) + && this.formatUploadCounts.equals(other.formatUploadCounts) + && this.formatUploadBytes.equals(other.formatUploadBytes) + && this.formatFailureCounts.equals(other.formatFailureCounts); } @Override @@ -577,7 +706,10 @@ public int hashCode() { uploadTimeMovingAverage, bytesLag, totalUploadTimeInMs, - directoryFileTransferTrackerStats + directoryFileTransferTrackerStats, + formatUploadCounts, + formatUploadBytes, + formatFailureCounts ); } @@ -626,6 +758,12 @@ public String toString() { + bytesLag + ", directoryFileTransferTrackerStats=" + directoryFileTransferTrackerStats + + ", formatUploadCounts=" + + formatUploadCounts + + ", formatUploadBytes=" + + formatUploadBytes + + ", formatFailureCounts=" + + formatFailureCounts + '}'; } } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 19f845991a8d8..41feeeae64f1e 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -144,11 +144,13 @@ import org.opensearch.index.engine.SafeCommitInfo; import org.opensearch.index.engine.Segment; import org.opensearch.index.engine.SegmentsStats; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.engine.exec.bridge.CheckpointState; import org.opensearch.index.engine.exec.bridge.Indexer; import org.opensearch.index.engine.exec.bridge.IndexingThrottler; import org.opensearch.index.engine.exec.bridge.StatsHolder; import org.opensearch.index.engine.exec.composite.CompositeDataFormatWriter; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.engine.exec.coord.CompositeEngine; import org.opensearch.index.fielddata.FieldDataStats; import org.opensearch.index.fielddata.ShardFieldData; @@ -182,13 +184,14 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.PrimaryReplicaSyncer.ResyncTask; import org.opensearch.index.similarity.SimilarityService; +import org.opensearch.index.store.CompositeStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteStoreFileDownloader; import org.opensearch.index.store.Store; import org.opensearch.index.store.Store.MetadataSnapshot; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.index.store.StoreStats; -import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.RemoteFsTranslog; import org.opensearch.index.translog.RemoteTranslogStats; @@ -221,6 +224,7 @@ import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationTimer; import org.opensearch.plugins.PluginsService; +import org.opensearch.plugins.SearchEnginePlugin; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.search.suggest.completion.CompletionStats; @@ -239,6 +243,7 @@ import java.util.Collection; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -345,6 +350,7 @@ Runnable getGlobalCheckpointSyncer() { private final ShardPath path; private final IndexShardOperationPermits indexShardOperationPermits; + private final PluginsService pluginsService; private static final EnumSet readAllowedStates = EnumSet.of(IndexShardState.STARTED, IndexShardState.POST_RECOVERY); // for primaries, we only allow to write when actually started (so the cluster has decided we started) @@ -401,7 +407,6 @@ Runnable getGlobalCheckpointSyncer() { private final MergedSegmentPublisher mergedSegmentPublisher; private final ReferencedSegmentsPublisher referencedSegmentsPublisher; private final Set pendingMergedSegmentCheckpoints = Sets.newConcurrentHashSet(); - private final PluginsService pluginsService; @InternalApi public IndexShard( final ShardRouting shardRouting, @@ -484,6 +489,8 @@ public IndexShard( state = IndexShardState.CREATED; this.path = path; this.circuitBreakerService = circuitBreakerService; + this.pluginsService = pluginsService; + /* create engine config */ logger.debug("state: [CREATED]"); @@ -562,7 +569,6 @@ public boolean shouldCache(Query query) { this.clusterApplierService = clusterApplierService; this.mergedSegmentPublisher = mergedSegmentPublisher; this.referencedSegmentsPublisher = referencedSegmentsPublisher; - this.pluginsService = pluginsService; synchronized (this.refreshMutex) { if (shardLevelRefreshEnabled) { startRefreshTask(); @@ -573,6 +579,22 @@ public boolean shouldCache(Query query) { public CompositeEngine getIndexingExecutionCoordinator() { return currentCompositeEngineReference.get(); } + + /** + * Complete the initialization of CompositeEngine refresh listeners after all dependencies are ready. + * This method should be called after remote store stats trackers have been created. + */ + public void completeCompositeEngineInitialization() { + try { + final EngineConfig engineConfig = newEngineConfig(() -> replicationTracker.getGlobalCheckpoint()); + getIndexingExecutionCoordinator().initializeRefreshListeners(engineConfig); + logger.debug("Completed CompositeEngine refresh listener initialization for shard [{}]", shardId); + } catch (IOException e) { + logger.error("Failed to complete CompositeEngine initialization for shard [{}]", shardId, e); + throw new RuntimeException("Failed to complete CompositeEngine initialization", e); + } + } + /** * By default, UNASSIGNED_SEQ_NO is used as the initial global checkpoint for new shard initialization. Ingestion * source does not track sequence numbers explicitly and hence defaults to NO_OPS_PERFORMED for compatibility. @@ -1820,6 +1842,40 @@ public void finalizeReplication(SegmentInfos infos) throws IOException { } } + /** + * Finalizes replication using CatalogSnapshot instead of SegmentInfos. + * This method replaces the SegmentInfos-based approach with CatalogSnapshot + * to ensure consistent version tracking and format-aware segment management. + * + * @param catalogSnapshot the CatalogSnapshot containing segment metadata + * @throws IOException if an error occurs during replication finalization + */ + public void finalizeReplication(CatalogSnapshot catalogSnapshot) throws IOException { + assert Thread.holdsLock(mutex) == false : "finalizeReplication must not be called under mutex"; + assert state == IndexShardState.RECOVERING || state == IndexShardState.STARTED; + + if (catalogSnapshot == null) { + logger.warn("Cannot finalize replication with null CatalogSnapshot for shard [{}]", shardId); + return; + } + + // Use the fresh catalogSnapshot parameter directly (like old SegmentInfos pattern) + try { + final ReplicationCheckpoint checkpoint = computeReplicationCheckpoint(catalogSnapshot); + replicationTracker.setLatestReplicationCheckpoint(checkpoint); + getIndexingExecutionCoordinator().setCatalogSnapshot(catalogSnapshot, this.shardPath()); + logger.trace("Updated replication checkpoint from fresh CatalogSnapshot: shard={}, checkpoint={}", shardId, checkpoint); + } catch (IOException e) { + logger.error("Error computing replication checkpoint from fresh catalog snapshot for shard [{}]", shardId, e); + throw new OpenSearchException("Error computing replication checkpoint from fresh catalog snapshot", e); + } + + logger.debug("Finalized replication with CatalogSnapshot: generation={}, version={}, shard={}", + catalogSnapshot.getGeneration(), catalogSnapshot.getVersion(), shardId); + getIndexingExecutionCoordinator().updateSearchEngine(); + + } + /** * The replica shard cleans up redundant pending merged segments based on the referenced segments of the primary shard. * Here, an example of generating redundant pending merged segments will be provided. @@ -1945,15 +2001,75 @@ public Tuple, ReplicationCheckpoint> getLatestSegme } /** - * Compute the latest {@link ReplicationCheckpoint} from a SegmentInfos. + * Compute the latest {@link ReplicationCheckpoint} from a CatalogSnapshot. * This function fetches a metadata snapshot from the store that comes with an IO cost. * We will reuse the existing stored checkpoint if it is at the same SI version. * - * @param segmentInfos {@link SegmentInfos} infos to use to compute. - * @return {@link ReplicationCheckpoint} Checkpoint computed from the infos. + * @param catalogSnapshot {@link CatalogSnapshot} snapshot to use to compute. + * @return {@link ReplicationCheckpoint} Checkpoint computed from the catalog snapshot. * @throws IOException When there is an error computing segment metadata from the store. * TODO: SegRep changes for decoupling. looks to depend on codec. */ + ReplicationCheckpoint computeReplicationCheckpoint(CatalogSnapshot catalogSnapshot) throws IOException { + if (catalogSnapshot == null) { + return ReplicationCheckpoint.empty(shardId); + } + final ReplicationCheckpoint latestReplicationCheckpoint = getLatestReplicationCheckpoint(); + if (latestReplicationCheckpoint.getSegmentInfosVersion() == catalogSnapshot.getVersion() + && latestReplicationCheckpoint.getSegmentsGen() == catalogSnapshot.getGeneration() + && latestReplicationCheckpoint.getPrimaryTerm() == getOperationPrimaryTerm()) { + return latestReplicationCheckpoint; + } + + // Extract format-aware metadata directly from CatalogSnapshot + final Map formatAwareMetadataMap = extractFormatAwareMetadata(catalogSnapshot); + + final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( + this.shardId, + getOperationPrimaryTerm(), + catalogSnapshot.getGeneration(), + catalogSnapshot.getVersion(), + formatAwareMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), + formatAwareMetadataMap, + getEngine().config().getCodec().getName() + ); + logger.trace("Recomputed ReplicationCheckpoint from CatalogSnapshot for shard {}", checkpoint); + return checkpoint; + } + + /** + * Extracts format-aware metadata from CatalogSnapshot. + * Creates a mapping from FileMetadata to StoreFileMetadata preserving format information. + */ + private Map extractFormatAwareMetadata(CatalogSnapshot catalogSnapshot) throws IOException { + Map formatAwareMap = new HashMap<>(); + + if(catalogSnapshot == null){ + return formatAwareMap; + } + + for (FileMetadata fileMetadata : catalogSnapshot.getFileMetadataList()) { + try { + long fileLength = store.compositeStoreDirectory().fileLength(fileMetadata); + long checksum = store.compositeStoreDirectory().calculateChecksum(fileMetadata); + + StoreFileMetadata storeFileMetadata = new StoreFileMetadata( + fileMetadata.file(), + fileLength, + Long.toString(checksum), + Version.LATEST, + fileMetadata.dataFormat() + ); + formatAwareMap.put(fileMetadata, storeFileMetadata); + } catch (IOException e) { + logger.warn("Failed to create StoreFileMetadata for {}", fileMetadata, e); + } + } + + logger.debug("Extracted {} format-aware metadata entries from CatalogSnapshot", formatAwareMap.size()); + return formatAwareMap; + } + ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) throws IOException { if (segmentInfos == null) { return ReplicationCheckpoint.empty(shardId); @@ -1965,7 +2081,7 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th return latestReplicationCheckpoint; } final Map metadataMap = store.getSegmentMetadataMap(segmentInfos); - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( + final ReplicationCheckpoint checkpoint = ReplicationCheckpoint.fromLegacyMetadata( this.shardId, getOperationPrimaryTerm(), segmentInfos.getGeneration(), @@ -2168,6 +2284,22 @@ public Map getSegmentMetadataMap() throws IOException } } + /** + * Fetch a format-aware map of FileMetadata to StoreFileMetadata from the latest CatalogSnapshot. + * This method preserves format information and is used for format-aware segment replication diffs. + * + * @return - Map of FileMetadata to StoreFileMetadata preserving format information + * @throws IOException - When there is an error loading metadata from the store. + */ + public Map getFormatAwareSegmentMetadataMap() throws IOException { + try (GatedCloseable catalogSnapshotRef = getCatalogSnapshotFromEngine()) { + return extractFormatAwareMetadata(catalogSnapshotRef.get()); + } catch (IOException e) { + logger.warn("Failed to get format-aware segment metadata", e); + return Collections.emptyMap(); + } + } + /** * Fails the shard and marks the shard store as corrupted if * e is caused by index corruption @@ -2375,7 +2507,7 @@ public boolean isRemoteSegmentStoreInSync() { try { RemoteSegmentStoreDirectory directory = getRemoteDirectory(); if (directory.readLatestMetadataFile() != null) { - Collection uploadFiles = directory.getSegmentsUploadedToRemoteStore().keySet(); + Collection uploadFiles = directory.getSegmentsUploadedToRemoteStore().keySet(); try (GatedCloseable segmentInfosGatedCloseable = getSegmentInfosSnapshot()) { Collection localSegmentInfosFiles = segmentInfosGatedCloseable.get().files(true); Set localFiles = new HashSet<>(localSegmentInfosFiles); @@ -4415,7 +4547,7 @@ private boolean hasOneRemoteSegmentSyncHappened() { RemoteSegmentStoreDirectory rd = getRemoteDirectory(); AtomicBoolean segment_n_uploaded = new AtomicBoolean(false); rd.getSegmentsUploadedToRemoteStore().forEach((key, value) -> { - if (key.startsWith("segments")) { + if (key.file().startsWith("segments")) { segment_n_uploaded.set(true); } }); @@ -5200,11 +5332,13 @@ public void afterRefresh(boolean didRefresh) throws IOException { } private void updateReplicationCheckpoint() { - final Tuple, ReplicationCheckpoint> tuple = getLatestSegmentInfosAndCheckpoint(); - try (final GatedCloseable ignored = tuple.v1()) { - replicationTracker.setLatestReplicationCheckpoint(tuple.v2()); + try (GatedCloseable catalogSnapshotRef = getCatalogSnapshotFromEngine()) { + final ReplicationCheckpoint checkpoint = computeReplicationCheckpoint(catalogSnapshotRef.get()); + replicationTracker.setLatestReplicationCheckpoint(checkpoint); + logger.trace("Updated replication checkpoint from CatalogSnapshot: shard={}, checkpoint={}", shardId, checkpoint); } catch (IOException e) { - throw new OpenSearchException("Error Closing SegmentInfos Snapshot", e); + logger.error("Error computing replication checkpoint from catalog snapshot for shard [{}]", shardId, e); + // throw new OpenSearchException("Error computing replication checkpoint from catalog snapshot", e); } } @@ -5428,11 +5562,11 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn // are uploaded to the remote segment store. RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); - Map uploadedSegments = remoteDirectory + Map uploadedSegments = remoteDirectory .getSegmentsUploadedToRemoteStore() .entrySet() .stream() - .filter(entry -> entry.getKey().startsWith(IndexFileNames.SEGMENTS) == false) + .filter(entry -> entry.getKey().file().startsWith(IndexFileNames.SEGMENTS) == false) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); store.incRef(); remoteStore.incRef(); @@ -5440,19 +5574,21 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn final Directory storeDirectory; if (recoveryState.getStage() == RecoveryState.Stage.INDEX) { storeDirectory = new StoreRecovery.StatsDirectoryWrapper(store.directory(), recoveryState.getIndex()); - for (String file : uploadedSegments.keySet()) { + for (FileMetadata file : uploadedSegments.keySet()) { long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { - recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), false); + recoveryState.getIndex().addFileDetail(file.file(), uploadedSegments.get(file).getLength(), false); } else { - recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), true); + recoveryState.getIndex().addFileDetail(file.file(), uploadedSegments.get(file).getLength(), true); } } } else { storeDirectory = store.directory(); } + if (indexSettings.isWarmIndex() == false) { - copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal, onFileSync); + // ToDo:@Kamal update while restore implementation + // copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal, onFileSync); } if (remoteSegmentMetadata != null) { @@ -5487,6 +5623,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn } /** + * ToDo: @Kamal, Implement this API during Restore flow * Downloads segments from given remote segment store for a specific commit. * @param overrideLocal flag to override local segment files with those in remote store * @param sourceRemoteDirectory RemoteSegmentDirectory Instance from which we need to sync segments @@ -5498,88 +5635,15 @@ public void syncSegmentsFromGivenRemoteSegmentStore( RemoteSegmentMetadata remoteSegmentMetadata, boolean pinnedTimestamp ) throws IOException { - logger.trace("Downloading segments from given remote segment store"); - RemoteSegmentStoreDirectory remoteDirectory = null; - if (remoteStore != null) { - remoteDirectory = getRemoteDirectory(); - remoteDirectory.init(); - remoteStore.incRef(); - } - Map uploadedSegments = sourceRemoteDirectory - .getSegmentsUploadedToRemoteStore(); - store.incRef(); - try { - final Directory storeDirectory; - if (recoveryState.getStage() == RecoveryState.Stage.INDEX) { - storeDirectory = new StoreRecovery.StatsDirectoryWrapper(store.directory(), recoveryState.getIndex()); - for (String file : uploadedSegments.keySet()) { - long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); - if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { - recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), false); - } else { - recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), true); - } - } - } else { - storeDirectory = store.directory(); - } - - String segmentsNFile = copySegmentFiles( - storeDirectory, - sourceRemoteDirectory, - remoteDirectory, - uploadedSegments, - overrideLocal, - () -> {} - ); - if (pinnedTimestamp) { - final SegmentInfos infosSnapshot = store.buildSegmentInfos( - remoteSegmentMetadata.getSegmentInfosBytes(), - remoteSegmentMetadata.getGeneration() - ); - long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); - // delete any other commits, we want to start the engine only from a new commit made with the downloaded infos bytes. - // Extra segments will be wiped on engine open. - for (String file : List.of(store.directory().listAll())) { - if (file.startsWith(IndexFileNames.SEGMENTS)) { - store.deleteQuiet(file); - } - } - assert Arrays.stream(store.directory().listAll()).filter(f -> f.startsWith(IndexFileNames.SEGMENTS)).findAny().isEmpty() - || indexSettings.isWarmIndex() : "There should not be any segments file in the dir"; - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); - } else if (segmentsNFile != null) { - try ( - ChecksumIndexInput indexInput = new BufferedChecksumIndexInput( - storeDirectory.openInput(segmentsNFile, IOContext.READONCE) - ) - ) { - long commitGeneration = SegmentInfos.generationFromSegmentsFileName(segmentsNFile); - SegmentInfos infosSnapshot = SegmentInfos.readCommit(store.directory(), indexInput, commitGeneration); - long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); - if (remoteStore != null) { - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); - } else { - store.directory().sync(infosSnapshot.files(true)); - store.directory().syncMetaData(); - } - } - } - } catch (IOException e) { - throw new IndexShardRecoveryException(shardId, "Exception while copying segment files from remote segment store", e); - } finally { - store.decRef(); - if (remoteStore != null) { - remoteStore.decRef(); - } - } + throw new UnsupportedOperationException("Not implemented yet"); } + // ToDo: Needs to be updated while Replication flow implementation private String copySegmentFiles( - Directory storeDirectory, + CompositeStoreDirectory storeDirectory, RemoteSegmentStoreDirectory sourceRemoteDirectory, RemoteSegmentStoreDirectory targetRemoteDirectory, - Map uploadedSegments, + Map uploadedSegments, boolean overrideLocal, final Runnable onFileSync ) throws IOException { @@ -5589,28 +5653,29 @@ private String copySegmentFiles( try { if (overrideLocal) { - for (String file : storeDirectory.listAll()) { + for (FileMetadata file : storeDirectory.listAll()) { storeDirectory.deleteFile(file); } } - for (String file : uploadedSegments.keySet()) { + for (FileMetadata file : uploadedSegments.keySet()) { long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { - toDownloadSegments.add(file); + toDownloadSegments.add(file.file()); } else { - skippedSegments.add(file); + skippedSegments.add(file.file()); } - if (file.startsWith(IndexFileNames.SEGMENTS)) { + if (file.file().startsWith(IndexFileNames.SEGMENTS)) { assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; - segmentNFile = file; + segmentNFile = file.file(); } } if (toDownloadSegments.isEmpty() == false) { try { - fileDownloader.download(sourceRemoteDirectory, storeDirectory, targetRemoteDirectory, toDownloadSegments, onFileSync); + // ToDo: @Kamal, Implement while restore flow implementation. + // fileDownloader.download(sourceRemoteDirectory, storeDirectory, targetRemoteDirectory, toDownloadSegments, onFileSync); } catch (Exception e) { throw new IOException("Error occurred when downloading segments from remote store", e); } @@ -5623,32 +5688,41 @@ private String copySegmentFiles( return segmentNFile; } - // Visible for testing - boolean localDirectoryContains(Directory localDirectory, String file, long checksum) throws IOException { - try (IndexInput indexInput = localDirectory.openInput(file, IOContext.READONCE)) { + // ToDo: @Kamal + boolean localDirectoryContains(CompositeStoreDirectory localDirectory, FileMetadata fileMetadata, long checksum) throws IOException { + throw new UnsupportedOperationException("Not implemented yet"); + } + + // ToDo: @Kamal + @Deprecated + boolean localDirectoryContains(Directory localDirectory, FileMetadata fileMetadata, long checksum) throws IOException { + try (IndexInput indexInput = localDirectory.openInput(fileMetadata.file(), IOContext.READONCE)) { if (checksum == CodecUtil.retrieveChecksum(indexInput)) { return true; } else { - logger.warn("Checksum mismatch between local and remote segment file: {}, will override local file", file); + logger.warn("Checksum mismatch between local and remote segment file: {}, will override local file", fileMetadata); // If there is a checksum mismatch and we are not serving reads it is safe to go ahead and delete the file now. // Outside of engine resets this method will be invoked during recovery so this is safe. if (isReadAllowed() == false) { - localDirectory.deleteFile(file); + localDirectory.deleteFile(fileMetadata.file()); } else { // segment conflict with remote store while the shard is serving reads. - failShard("Local copy of segment " + file + " has a different checksum than the version in remote store", null); + failShard("Local copy of segment " + fileMetadata.file() + " has a different checksum than the version in remote store", null); } } } catch (NoSuchFileException | FileNotFoundException e) { - logger.debug("File {} does not exist in local FS, downloading from remote store", file); + logger.debug("File {} does not exist in local FS, downloading from remote store", fileMetadata.file()); } catch (IOException e) { - logger.warn("Exception while reading checksum of file: {}, this can happen if file is corrupted", file); + logger.warn("Exception while reading checksum of file: {}, this can happen if file is corrupted", fileMetadata.file()); // For any other exception on reading checksum, we delete the file to re-download again - localDirectory.deleteFile(file); + localDirectory.deleteFile(fileMetadata.file()); } + return false; } + + /** * Returns the maximum sequence number of either update or delete operations have been processed in this shard * or the sequence number from {@link #advanceMaxSeqNoOfUpdatesOrDeletes(long)}. An index request is considered @@ -5704,6 +5778,15 @@ public GatedCloseable getSegmentInfosSnapshot() { return getEngine().getSegmentInfosSnapshot(); } + public GatedCloseable getCatalogSnapshotFromEngine() { + try { + return getIndexingExecutionCoordinator().getCatalogSnapshotReference(); + } + catch (Exception e) { + throw new OpenSearchException("Error occurred while getting catalog snapshot", e); + } + } + private TimeValue getRemoteTranslogUploadBufferInterval(Supplier clusterRemoteTranslogBufferIntervalSupplier) { assert Objects.nonNull(clusterRemoteTranslogBufferIntervalSupplier) : "remote translog buffer interval supplier is null"; if (indexSettings().isRemoteTranslogBufferIntervalExplicit()) { diff --git a/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java index 80daefc4482fc..4844c334e13d2 100644 --- a/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java @@ -14,6 +14,8 @@ import org.opensearch.common.lease.Releasables; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.index.engine.CatalogSnapshotAwareRefreshListener; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -28,7 +30,7 @@ * is called, all the permits are acquired and there are no available permits to afterRefresh. This abstract class provides * necessary abstract methods to schedule retry. */ -public abstract class ReleasableRetryableRefreshListener implements ReferenceManager.RefreshListener { +public abstract class ReleasableRetryableRefreshListener implements ReferenceManager.RefreshListener, CatalogSnapshotAwareRefreshListener { /** * Total permits = 1 ensures that there is only single instance of runAfterRefreshWithPermit that is running at a time. @@ -58,11 +60,22 @@ public ReleasableRetryableRefreshListener(ThreadPool threadPool) { this.threadPool = threadPool; } + @Override + public final void afterRefresh(boolean didRefresh, CatalogSnapshot catalogSnapshot) throws IOException { + if (closed.get()) { + return; + } + + runAfterRefreshExactlyOnce(didRefresh); + runAfterRefreshWithPermit(didRefresh, () -> {}); + } + @Override public final void afterRefresh(boolean didRefresh) throws IOException { if (closed.get()) { return; } + runAfterRefreshExactlyOnce(didRefresh); runAfterRefreshWithPermit(didRefresh, () -> {}); } 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 f3ee23a7505d2..bfbb10b1da5e0 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -9,12 +9,8 @@ package org.opensearch.index.shard; import org.apache.logging.log4j.Logger; -import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; import org.opensearch.action.LatchedActionListener; import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.cluster.routing.RecoverySource; @@ -25,8 +21,11 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.index.engine.EngineException; import org.opensearch.index.engine.InternalEngine; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.remote.RemoteSegmentTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.CompositeStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.Translog; @@ -42,6 +41,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -82,7 +82,7 @@ public final class RemoteStoreRefreshListener extends ReleasableRetryableRefresh public static final Set EXCLUDE_FILES = Set.of("write.lock"); private final IndexShard indexShard; - private final Directory storeDirectory; + private final CompositeStoreDirectory compositeStoreDirectory; private final RemoteSegmentStoreDirectory remoteDirectory; private final RemoteSegmentTransferTracker segmentTracker; private final Map localSegmentChecksumMap; @@ -101,10 +101,10 @@ public RemoteStoreRefreshListener( super(indexShard.getThreadPool()); logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.indexShard = indexShard; - this.storeDirectory = indexShard.store().directory(); + this.compositeStoreDirectory = indexShard.store().compositeStoreDirectory(); this.remoteDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()) .getDelegate()).getDelegate(); - remoteStoreUploader = new RemoteStoreUploaderService(indexShard, storeDirectory, remoteDirectory); + remoteStoreUploader = new RemoteStoreUploaderService(indexShard, compositeStoreDirectory, remoteDirectory); localSegmentChecksumMap = new HashMap<>(); RemoteSegmentMetadata remoteSegmentMetadata = null; if (indexShard.routingEntry().primary()) { @@ -134,9 +134,9 @@ protected void runAfterRefreshExactlyOnce(boolean didRefresh) { if (shouldSync(didRefresh, true) && isReadyForUpload()) { try { segmentTracker.updateLocalRefreshTimeAndSeqNo(); - try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { - Collection localSegmentsPostRefresh = segmentInfosGatedCloseable.get().files(true); - updateLocalSizeMapAndTracker(localSegmentsPostRefresh); + try (GatedCloseable catalogSnapshotRef = indexShard.getCatalogSnapshotFromEngine()) { + Collection localFilesPostRefresh = catalogSnapshotRef.get().getFileMetadataList(); + updateLocalSizeMapAndTracker(localFilesPostRefresh); } } catch (Throwable t) { logger.error("Exception in runAfterRefreshExactlyOnce() method", t); @@ -203,8 +203,9 @@ private boolean shouldSync(boolean didRefresh, boolean skipPrimaryTermCheck) { * @return true iff all the local files are uploaded to remote store. */ boolean isRemoteSegmentStoreInSync() { - try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { - return segmentInfosGatedCloseable.get().files(true).stream().allMatch(this::skipUpload); + try (GatedCloseable catalogSnapshotRef = indexShard.getCatalogSnapshotFromEngine()) { + Collection localFiles = catalogSnapshotRef.get().getFileMetadataList(); + return localFiles.stream().allMatch(this::skipUpload); } catch (Throwable throwable) { logger.error("Throwable thrown during isRemoteSegmentStoreInSync", throwable); } @@ -235,13 +236,14 @@ private boolean syncSegments() { // if a new segments_N file is present in local that is not uploaded to remote store yet, it // is considered as a first refresh post commit. A cleanup of stale commit files is triggered. // This is done to avoid delete post each refresh. + // ToDo: @Kamal, Update while implementing Snapshot restore if (isRefreshAfterCommit()) { remoteDirectory.deleteStaleSegmentsAsync(indexShard.getRemoteStoreSettings().getMinRemoteSegmentMetadataFiles()); } - try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { - SegmentInfos segmentInfos = segmentInfosGatedCloseable.get(); - final ReplicationCheckpoint checkpoint = indexShard.computeReplicationCheckpoint(segmentInfos); + try (GatedCloseable catalogSnapshotRef = indexShard.getCatalogSnapshotFromEngine()) { + CatalogSnapshot catalogSnapshot = catalogSnapshotRef.get(); + final ReplicationCheckpoint checkpoint = indexShard.computeReplicationCheckpoint(catalogSnapshot); if (checkpoint.getPrimaryTerm() != indexShard.getOperationPrimaryTerm()) { throw new IllegalStateException( String.format( @@ -255,12 +257,21 @@ private boolean syncSegments() { // Capture replication checkpoint before uploading the segments as upload can take some time and checkpoint can // move. long lastRefreshedCheckpoint = ((InternalEngine) indexShard.getEngine()).lastRefreshedCheckpoint(); - Collection localSegmentsPostRefresh = segmentInfos.files(true); - // Create a map of file name to size and update the refresh segment tracker - Map localSegmentsSizeMap = updateLocalSizeMapAndTracker(localSegmentsPostRefresh).entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Collection localFilesPostRefresh = catalogSnapshot.getFileMetadataList(); + + // Log format-aware statistics + Map formatCounts = localFilesPostRefresh.stream() + .collect(Collectors.groupingBy( + fm -> fm.dataFormat(), + Collectors.counting() + )); + + logger.debug("Format-aware segment upload initiated: totalFiles={}, formatBreakdown={}", + localFilesPostRefresh.size(), formatCounts); + + Map fileMetadataToSizeMap = updateLocalSizeMapAndTracker(localFilesPostRefresh); + CountDownLatch latch = new CountDownLatch(1); ActionListener segmentUploadsCompletedListener = new LatchedActionListener<>(new ActionListener<>() { @Override @@ -268,15 +279,15 @@ public void onResponse(Void unused) { try { logger.debug("New segments upload successful"); // Start metadata file upload - uploadMetadata(localSegmentsPostRefresh, segmentInfos, checkpoint); + uploadMetadata(localFilesPostRefresh, catalogSnapshot, checkpoint); logger.debug("Metadata upload successful"); - clearStaleFilesFromLocalSegmentChecksumMap(localSegmentsPostRefresh); + clearStaleFilesFromLocalSegmentChecksumMap(localFilesPostRefresh); onSuccessfulSegmentsSync( refreshTimeMs, refreshClockTimeMs, refreshSeqNo, lastRefreshedCheckpoint, - localSegmentsSizeMap, + fileMetadataToSizeMap, checkpoint ); // At this point since we have uploaded new segments, segment infos and segment metadata file, @@ -296,18 +307,17 @@ public void onFailure(Exception e) { } }, latch); - // Start the segments files upload - uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); + // Start the segments files upload using FileMetadata + uploadNewSegments(localFilesPostRefresh, fileMetadataToSizeMap, segmentUploadsCompletedListener); if (latch.await( remoteStoreSettings.getClusterRemoteSegmentTransferTimeout().millis(), TimeUnit.MILLISECONDS ) == false) { throw new SegmentUploadFailedException("Timeout while waiting for remote segment transfer to complete"); } - } catch (EngineException e) { - logger.warn("Exception while reading SegmentInfosSnapshot", e); } - } catch (IOException e) { + } + catch (IOException 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 // with remote trans-log integration. @@ -326,23 +336,48 @@ public void onFailure(Exception e) { /** * Uploads new segment files to the remote store. * - * @param localSegmentsPostRefresh collection of segment files present after refresh - * @param localSegmentsSizeMap map of segment file names to their sizes + * @param localFilesPostRefresh collection of FileMetadata objects containing format and file information + * @param localFileMetadataSizeMap map of segment file names to their sizes * @param segmentUploadsCompletedListener listener to be notified when upload completes */ private void uploadNewSegments( - Collection localSegmentsPostRefresh, - Map localSegmentsSizeMap, + Collection localFilesPostRefresh, + Map localFileMetadataSizeMap, ActionListener segmentUploadsCompletedListener ) { - Collection filteredFiles = localSegmentsPostRefresh.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); - Function, UploadListener> uploadListenerFunction = (Map sizeMap) -> createUploadListener( - localSegmentsSizeMap + // Filter FileMetadata objects based on whether their files should be uploaded + Collection filteredFileMetadata = localFilesPostRefresh.stream() + .filter(fileMetadata -> !skipUpload(fileMetadata)) + .collect(Collectors.toList()); + + // Log format-aware upload statistics + Map uploadFormatCounts = filteredFileMetadata.stream() + .collect(Collectors.groupingBy( + fm -> fm.dataFormat(), + Collectors.counting() + )); + + Map skippedFormatCounts = localFilesPostRefresh.stream() + .filter(fileMetadata -> skipUpload(fileMetadata)) + .collect(Collectors.groupingBy( + fm -> fm.dataFormat(), + Collectors.counting() + )); + + logger.debug("Format-aware upload filtering: totalFiles={}, uploadFiles={}, skippedFiles={}, " + + "uploadFormats={}, skippedFormats={}", + localFilesPostRefresh.size(), filteredFileMetadata.size(), + localFilesPostRefresh.size() - filteredFileMetadata.size(), + uploadFormatCounts, skippedFormatCounts); + + Function, UploadListener> uploadListenerFunction = (Map sizeMap) -> createUploadListener( + localFileMetadataSizeMap ); + // Pass FileMetadata collection to RemoteStoreUploaderService remoteStoreUploader.uploadSegments( - filteredFiles, - localSegmentsSizeMap, + filteredFileMetadata, + localFileMetadataSizeMap, segmentUploadsCompletedListener, uploadListenerFunction, isLowPriorityUpload() @@ -352,19 +387,20 @@ private void uploadNewSegments( /** * Clears the stale files from the latest local segment checksum map. * - * @param localSegmentsPostRefresh list of segment files present post refresh + * @param localSegmentsPostRefresh collection of FileMetadata for files present post refresh */ - private void clearStaleFilesFromLocalSegmentChecksumMap(Collection localSegmentsPostRefresh) { - localSegmentChecksumMap.keySet() - .stream() - .filter(file -> !localSegmentsPostRefresh.contains(file)) - .collect(Collectors.toSet()) - .forEach(localSegmentChecksumMap::remove); + private void clearStaleFilesFromLocalSegmentChecksumMap(Collection localSegmentsPostRefresh) { + Set currentFileNames = localSegmentsPostRefresh.stream() + .map(FileMetadata::file) + .collect(Collectors.toSet()); + localSegmentChecksumMap.keySet().removeIf(key -> !currentFileNames.contains(key)); } private void beforeSegmentsSync() { - // Start tracking total uploads started - segmentTracker.incrementTotalUploadsStarted(); + if(segmentTracker != null) { + // Start tracking total uploads started + segmentTracker.incrementTotalUploadsStarted(); + } } private void onSuccessfulSegmentsSync( @@ -372,7 +408,7 @@ private void onSuccessfulSegmentsSync( long refreshClockTimeMs, long refreshSeqNo, long lastRefreshedCheckpoint, - Map localFileSizeMap, + Map localFileSizeMap, ReplicationCheckpoint checkpoint ) { // Update latest uploaded segment files name in segment tracker @@ -406,9 +442,11 @@ protected String getRetryThreadPoolName() { } private boolean isRefreshAfterCommit() throws IOException { - String lastCommittedLocalSegmentFileName = SegmentInfos.getLastCommitSegmentsFileName(storeDirectory); - return (lastCommittedLocalSegmentFileName != null - && !remoteDirectory.containsFile(lastCommittedLocalSegmentFileName, getChecksumOfLocalFile(lastCommittedLocalSegmentFileName))); + // ToDo:@Kamal Get last commit generation from catalogSnapshot + // String lastCommittedLocalSegmentFileName = SegmentInfos.getLastCommitSegmentsFileName(compositeStoreDirectory); +// return (lastCommittedLocalSegmentFileName != null +// && !remoteDirectory.containsFile(lastCommittedLocalSegmentFileName, getChecksumOfLocalFile(lastCommittedLocalSegmentFileName))); + return true; } /** @@ -424,14 +462,15 @@ private boolean isRefreshAfterCommitSafe() { return false; } - void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos segmentInfos, ReplicationCheckpoint replicationCheckpoint) + // ToDo:@Kamal Update MaxSeqNo + void uploadMetadata(Collection localFilesPostRefresh, CatalogSnapshot catalogSnapshot, ReplicationCheckpoint replicationCheckpoint) throws IOException { final long maxSeqNo = ((InternalEngine) indexShard.getEngine()).currentOngoingRefreshCheckpoint(); - SegmentInfos segmentInfosSnapshot = segmentInfos.clone(); - Map userData = segmentInfosSnapshot.getUserData(); + CatalogSnapshot catalogSnapshotCopy = catalogSnapshot.clone(); + Map userData = catalogSnapshotCopy.getUserData(); userData.put(LOCAL_CHECKPOINT_KEY, String.valueOf(maxSeqNo)); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); - segmentInfosSnapshot.setUserData(userData, false); + catalogSnapshotCopy.setUserData(userData, false); Translog.TranslogGeneration translogGeneration = indexShard.getEngine().translogManager().getTranslogGeneration(); if (translogGeneration == null) { @@ -439,9 +478,9 @@ void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos se } else { long translogFileGeneration = translogGeneration.translogFileGeneration; remoteDirectory.uploadMetadata( - localSegmentsPostRefresh, - segmentInfosSnapshot, - storeDirectory, + localFilesPostRefresh, + catalogSnapshotCopy, + compositeStoreDirectory, translogFileGeneration, replicationCheckpoint, indexShard.getNodeId() @@ -456,30 +495,37 @@ boolean isLowPriorityUpload() { /** * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. * - * @param file that needs to be uploaded. + * @param fileMetadata that needs to be uploaded. * @return true if the upload has to be skipped for the file. */ - private boolean skipUpload(String file) { + private boolean skipUpload(FileMetadata fileMetadata) { try { // Exclude files that are already uploaded and the exclude files to come up with the list of files to be uploaded. - return EXCLUDE_FILES.contains(file) || remoteDirectory.containsFile(file, getChecksumOfLocalFile(file)); + return EXCLUDE_FILES.contains(fileMetadata.file()) || remoteDirectory.containsFile(fileMetadata, getChecksumOfLocalFile(fileMetadata)); } catch (IOException e) { logger.error( - "Exception while reading checksum of local segment file: {}, ignoring the exception and re-uploading the file", - file + "Exception while reading checksum of local segment file: {}, format: {}, ignoring the exception and re-uploading the file", + fileMetadata.file(), fileMetadata.dataFormat() ); } return false; } - private String getChecksumOfLocalFile(String file) throws IOException { - if (!localSegmentChecksumMap.containsKey(file)) { - try (IndexInput indexInput = storeDirectory.openInput(file, IOContext.READONCE)) { - String checksum = Long.toString(CodecUtil.retrieveChecksum(indexInput)); - localSegmentChecksumMap.put(file, checksum); + private String getChecksumOfLocalFile(FileMetadata fileMetadata) throws IOException { + if (!localSegmentChecksumMap.containsKey(fileMetadata.file())) { + try{ + String checksum = Long.toString(compositeStoreDirectory.calculateChecksum(fileMetadata)); + localSegmentChecksumMap.put(fileMetadata.file(), checksum); + logger.debug("Calculated checksum for file: {}, format: {}, checksum: {}", + fileMetadata.file(), fileMetadata.dataFormat(), checksum); + } + catch (IOException e){ + logger.error("Failed to calculate checksum for file: {}, format: {}", + fileMetadata.file(), fileMetadata.dataFormat(), e); + throw e; } } - return localSegmentChecksumMap.get(file); + return localSegmentChecksumMap.get(fileMetadata.file()); } /** @@ -492,14 +538,30 @@ private void updateRemoteRefreshTimeAndSeqNo(long refreshTimeMs, long refreshClo } /** - * Updates map of file name to size of the input segment files in the segment tracker. Uses {@code storeDirectory.fileLength(file)} to get the size. + * Updates map of FileMetadata to size of the input segment files in the segment tracker. + * Uses CompositeStoreDirectory.fileLength(FileMetadata) for efficient format-aware file size retrieval. * - * @param segmentFiles list of segment files that are part of the most recent local refresh. + * @param localFilesPostRefresh collection of FileMetadata for files that are part of the most recent local refresh. * - * @return updated map of local segment files and filesize + * @return updated map of FileMetadata to file size */ - private Map updateLocalSizeMapAndTracker(Collection segmentFiles) { - return segmentTracker.updateLatestLocalFileNameLengthMap(segmentFiles, storeDirectory::fileLength); + private Map updateLocalSizeMapAndTracker(Collection localFilesPostRefresh) { + Map fileSizeMap = new HashMap<>(); + + for (FileMetadata fileMetadata : localFilesPostRefresh) { + try { + long fileSize = compositeStoreDirectory.fileLength(fileMetadata); + fileSizeMap.put(fileMetadata, fileSize); + } catch (IOException e) { + logger.warn("Failed to get file length for file: {}, format: {}", + fileMetadata.file(), fileMetadata.dataFormat(), e); + } + } + + // Update segment tracker with FileMetadata-based map + segmentTracker.updateLatestLocalFileNameLengthMap(fileSizeMap); + + return fileSizeMap; } private void updateFinalStatusInSegmentTracker(boolean uploadStatus, long bytesBeforeUpload, long startTimeInNS) { @@ -578,29 +640,29 @@ boolean isLocalOrSnapshotRecoveryOrSeeding() { * * @param fileSizeMap updated map of current snapshot of local segments to their sizes */ - private UploadListener createUploadListener(Map fileSizeMap) { + private UploadListener createUploadListener(Map fileSizeMap) { return new UploadListener() { private long uploadStartTime = 0; @Override - public void beforeUpload(String file) { + public void beforeUpload(FileMetadata fileMetadata) { // Start tracking the upload bytes started - segmentTracker.addUploadBytesStarted(fileSizeMap.get(file)); + segmentTracker.addUploadBytesStarted(fileSizeMap.get(fileMetadata)); uploadStartTime = System.currentTimeMillis(); } @Override - public void onSuccess(String file) { + public void onSuccess(FileMetadata fileMetadata) { // Track upload success - segmentTracker.addUploadBytesSucceeded(fileSizeMap.get(file)); - segmentTracker.addToLatestUploadedFiles(file); + segmentTracker.addUploadBytesSucceeded(fileSizeMap.get(fileMetadata)); + segmentTracker.addToLatestUploadedFiles(fileMetadata); segmentTracker.addUploadTimeInMillis(Math.max(1, System.currentTimeMillis() - uploadStartTime)); } @Override - public void onFailure(String file) { + public void onFailure(FileMetadata fileMetadata) { // Track upload failure - segmentTracker.addUploadBytesFailed(fileSizeMap.get(file)); + segmentTracker.addUploadBytesFailed(fileSizeMap.get(fileMetadata)); segmentTracker.addUploadTimeInMillis(Math.max(1, System.currentTimeMillis() - uploadStartTime)); } }; diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploader.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploader.java index 1a60aabf3e609..de2978ee2e7a8 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploader.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploader.java @@ -10,6 +10,7 @@ import org.opensearch.common.util.UploadListener; import org.opensearch.core.action.ActionListener; +import org.opensearch.index.engine.exec.FileMetadata; import java.util.Collection; import java.util.Map; @@ -20,11 +21,20 @@ */ public interface RemoteStoreUploader { + /** + * Upload segments using FileMetadata for format-aware routing + * + * @param fileMetadataCollection collection of FileMetadata objects containing format and file information + * @param fileMeatadataSizeMap map of segment file names to their sizes + * @param listener listener to be notified when upload completes + * @param uploadListenerFunction function to create upload listeners + * @param isLowPriorityUpload whether this is a low priority upload + */ void uploadSegments( - Collection localSegments, - Map localSegmentsSizeMap, + Collection fileMetadataCollection, + Map fileMeatadataSizeMap, ActionListener listener, - Function, UploadListener> uploadListenerFunction, + Function, UploadListener> uploadListenerFunction, boolean isLowPriorityUpload ); } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploaderService.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploaderService.java index 7cab0b258f107..651b447eddd29 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploaderService.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreUploaderService.java @@ -18,12 +18,15 @@ import org.opensearch.common.logging.Loggers; import org.opensearch.common.util.UploadListener; import org.opensearch.core.action.ActionListener; -import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.store.SegmentUploadFailedException; +import org.opensearch.index.store.CompositeStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import java.util.Collection; import java.util.Map; import java.util.function.Function; +import java.util.stream.Collectors; /** * The service essentially acts as a bridge between local segment storage and remote storage, @@ -34,56 +37,93 @@ public class RemoteStoreUploaderService implements RemoteStoreUploader { private final Logger logger; private final IndexShard indexShard; - private final Directory storeDirectory; + private final CompositeStoreDirectory storeDirectory; private final RemoteSegmentStoreDirectory remoteDirectory; + // Todo: Remove public RemoteStoreUploaderService(IndexShard indexShard, Directory storeDirectory, RemoteSegmentStoreDirectory remoteDirectory) { logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.indexShard = indexShard; - this.storeDirectory = storeDirectory; + this.storeDirectory = null; + this.remoteDirectory = remoteDirectory; + } + + public RemoteStoreUploaderService(IndexShard indexShard, CompositeStoreDirectory storeDirectory, RemoteSegmentStoreDirectory remoteDirectory) { + logger = Loggers.getLogger(getClass(), indexShard.shardId()); + this.indexShard = indexShard; + this.storeDirectory = storeDirectory; this.remoteDirectory = remoteDirectory; } @Override public void uploadSegments( - Collection localSegments, - Map localSegmentsSizeMap, + Collection fileMetadataCollection, + Map fileMetadataSizeMap, ActionListener listener, - Function, UploadListener> uploadListenerFunction, + Function, UploadListener> uploadListenerFunction, boolean isLowPriorityUpload ) { - if (localSegments.isEmpty()) { + if (fileMetadataCollection.isEmpty()) { logger.debug("No new segments to upload in uploadNewSegments"); listener.onResponse(null); return; } - logger.debug("Effective new segments files to upload {}", localSegments); + // Log format-aware upload statistics + Map formatCounts = fileMetadataCollection.stream() + .collect(Collectors.groupingBy( + fm -> fm.dataFormat(), + Collectors.counting() + )); + + Map formatSizes = fileMetadataCollection.stream() + .collect(Collectors.groupingBy( + fm -> fm.dataFormat(), + Collectors.summingLong(fm -> fileMetadataSizeMap.getOrDefault(fm.file(), 0L)) + )); + + logger.debug("Format-aware segment upload starting: totalFiles={}, formatCounts={}, formatSizes={}", + fileMetadataCollection.size(), formatCounts, formatSizes); ActionListener> mappedListener = ActionListener.map(listener, resp -> null); - GroupedActionListener batchUploadListener = new GroupedActionListener<>(mappedListener, localSegments.size()); - Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); + GroupedActionListener batchUploadListener = new GroupedActionListener<>(mappedListener, fileMetadataCollection.size()); + + CompositeStoreDirectory directory = storeDirectory; - for (String localSegment : localSegments) { + for (FileMetadata fileMetadata : fileMetadataCollection) { + String fileName = fileMetadata.file(); // Initializing listener here to ensure that the stats increment operations are thread-safe - UploadListener statsListener = uploadListenerFunction.apply(localSegmentsSizeMap); + UploadListener statsListener = uploadListenerFunction.apply(fileMetadataSizeMap); ActionListener aggregatedListener = ActionListener.wrap(resp -> { - statsListener.onSuccess(localSegment); + statsListener.onSuccess(fileMetadata); batchUploadListener.onResponse(resp); + + // Log format-specific upload success + long fileSize = fileMetadataSizeMap.getOrDefault(fileMetadata, 0L); + logger.debug("Format-aware upload completed: file={}, format={}, size={} bytes", + fileName, fileMetadata.dataFormat(), fileSize); + // Once uploaded to Remote, local files become eligible for eviction from FileCache - if (directory instanceof CompositeDirectory) { - ((CompositeDirectory) directory).afterSyncToRemote(localSegment); - } + // Todo:@Kamal Update compositeDirectory for ultrawarm support +// if (directory instanceof CompositeDirectory) { +// ((CompositeDirectory) directory).afterSyncToRemote(fileName); +// } }, ex -> { logger.warn(() -> new ParameterizedMessage("Exception: [{}] while uploading segment files", ex), ex); if (ex instanceof CorruptIndexException) { indexShard.failShard(ex.getMessage(), ex); + } else if (ex instanceof SegmentUploadFailedException) { + logger.error("Segment upload failed: file={}, format={}, error={}", + fileName, fileMetadata.dataFormat(), ex.getMessage()); + } else { + logger.warn("Unexpected upload failure: file={}, format={}, error={}", + fileName, fileMetadata.dataFormat(), ex.getMessage(), ex); } - statsListener.onFailure(localSegment); + + statsListener.onFailure(fileMetadata); batchUploadListener.onFailure(ex); }); - statsListener.beforeUpload(localSegment); - // Place where the actual upload is happening - remoteDirectory.copyFrom(storeDirectory, localSegment, IOContext.DEFAULT, aggregatedListener, isLowPriorityUpload); + statsListener.beforeUpload(fileMetadata); + remoteDirectory.copyFrom(fileMetadata, storeDirectory, IOContext.DEFAULT, aggregatedListener, isLowPriorityUpload); } } } diff --git a/server/src/main/java/org/opensearch/index/shard/ShardPath.java b/server/src/main/java/org/opensearch/index/shard/ShardPath.java index 911bfec94e190..5b23c1befefcf 100644 --- a/server/src/main/java/org/opensearch/index/shard/ShardPath.java +++ b/server/src/main/java/org/opensearch/index/shard/ShardPath.java @@ -59,7 +59,7 @@ */ @PublicApi(since = "1.0.0") public final class ShardPath { - public static final String INDEX_FOLDER_NAME = "index"; + public static final String INDEX_FOLDER_NAME = "lucene"; public static final String TRANSLOG_FOLDER_NAME = "translog"; private final Path 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 cb95c31941a0e..7f75c75fb8236 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -57,6 +57,7 @@ public class CompositeDirectory extends FilterDirectory { private static final Logger logger = LogManager.getLogger(CompositeDirectory.class); protected final FSDirectory localDirectory; + protected final CompositeStoreDirectory compositeStoreDirectory; protected final RemoteSegmentStoreDirectory remoteDirectory; protected final FileCache fileCache; protected final TransferManager transferManager; @@ -72,6 +73,7 @@ public CompositeDirectory(Directory localDirectory, Directory remoteDirectory, F super(localDirectory); validate(localDirectory, remoteDirectory, fileCache); this.localDirectory = (FSDirectory) localDirectory; + this.compositeStoreDirectory = null; this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; this.fileCache = fileCache; this.threadPool = threadPool; diff --git a/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectory.java new file mode 100644 index 0000000000000..1a8fa25ee2685 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectory.java @@ -0,0 +1,285 @@ +/* + * 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.logging.log4j.Logger; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.Any; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.plugins.DataSourcePlugin; +import org.opensearch.plugins.PluginsService; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Composite directory that coordinates multiple format-specific directories. + * Routes file operations to appropriate format directories based on file type. + * Implements both Directory and FormatStoreDirectory interfaces for compatibility. + * + * Follows the same plugin-based architecture pattern as CompositeIndexingExecutionEngine. + * + * @opensearch.api + */ +@PublicApi(since = "3.0.0") +public class CompositeStoreDirectory { + + private Any dataFormat; + private final Path directoryPath; + public final List> delegates = new ArrayList<>(); + public final HashMap> delegatesMap = new HashMap<>(); + + private final Logger logger; + private final DirectoryFileTransferTracker directoryFileTransferTracker; + private final ShardPath shardPath; + + /** + * Simplified constructor for auto-discovery (like CompositeIndexingExecutionEngine) + */ + public CompositeStoreDirectory(IndexSettings indexSettings, PluginsService pluginsService, ShardPath shardPath, Logger logger) { + this.shardPath = shardPath; + this.logger = logger; + this.directoryFileTransferTracker = new DirectoryFileTransferTracker(); + this.directoryPath = shardPath.getDataPath(); + + try { + DataSourcePlugin plugin = pluginsService.filterPlugins(DataSourcePlugin.class).stream() + .findAny() + .orElseThrow(() -> new IllegalArgumentException("dataformat is not registered.")); + delegates.add(plugin.createFormatStoreDirectory(indexSettings, shardPath)); + delegatesMap.put(plugin.getDataFormat().name(), delegates.get(delegates.size() - 1)); + } catch (NullPointerException | IOException e) { + throw new RuntimeException("Failed to create fallback directory", e); + } + } + + public void initialize() throws IOException { + // Initialize all delegates + for (FormatStoreDirectory delegate : delegates) { + delegate.initialize(); + } + } + public void cleanup() throws IOException { + // Cleanup all delegates + for (FormatStoreDirectory delegate : delegates) { + delegate.cleanup(); + } + } + + /** + * Gets the FormatStoreDirectory for a specific data format by name. + * Uses the delegatesMap to find the appropriate directory delegate. + * + * @param dataFormatName the name of the data format (e.g., "lucene", "parquet", "text") + * @return the FormatStoreDirectory that handles the specified format + * @throws IllegalArgumentException if no directory is found for the format + */ + public FormatStoreDirectory getDirectoryForFormat(String dataFormatName) { + logger.trace("Format routing request: searching for directory for format '{}'", dataFormatName); + + FormatStoreDirectory directory = delegatesMap.get(dataFormatName); + + if (directory == null) { + + if(dataFormatName.equalsIgnoreCase("TempMetadata") && !delegates.isEmpty()) + { + return delegates.getFirst(); + } + List availableFormats = new ArrayList<>(delegatesMap.keySet()); + + logger.error("Format routing failed: requested format '{}' not found. Available formats: {}. " + + "This indicates a configuration issue or missing format plugin.", + dataFormatName, availableFormats); + + throw new IllegalArgumentException( + String.format("No directory found for format '%s'. Available formats: %s", + dataFormatName, availableFormats) + ); + } + + logger.debug("Format routing successful: format '{}' routed to directory type '{}'", + dataFormatName, directory.getClass().getSimpleName()); + + return directory; + } + + // Directory interface implementation with routing + public FileMetadata[] listAll() throws IOException { + Set allFiles = new HashSet<>(); + for (FormatStoreDirectory directory : delegates) { + allFiles.addAll(Arrays.asList(directory.listAll())); + } + return allFiles.toArray(new FileMetadata[0]); + } + + public void sync(Collection fileMetadataList) throws IOException { + // Group files by directory and sync each directory + Map> filesByDirectory = new HashMap<>(); + + for (var fileMetadata : fileMetadataList) { + FormatStoreDirectory directory = getDirectoryForFormat(fileMetadata.dataFormat()); + filesByDirectory.computeIfAbsent(directory, k -> new ArrayList<>()).add(fileMetadata.file()); + } + + for (Map.Entry> entry : filesByDirectory.entrySet()) { + entry.getKey().sync(entry.getValue()); + } + } + + + public long getChecksumOfLocalFile(FileMetadata fileMetadata) throws IOException { + logger.debug("Getting checksum of local file: {}", fileMetadata.file()); + return calculateChecksum(fileMetadata); + } + + /** + * Returns the byte length of a file using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @return the length of the file in bytes + * @throws IOException if the file cannot be accessed + */ + public long fileLength(FileMetadata fileMetadata) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + return formatDirectory.fileLength(fileMetadata.file()); + } + + /** + * Deletes a file using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @throws IOException if the file exists but could not be deleted + */ + public void deleteFile(FileMetadata fileMetadata) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + formatDirectory.deleteFile(fileMetadata.file()); + } + + /** + * Opens an IndexInput for reading using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @param context IOContext providing performance hints for the operation + * @return IndexInput for reading from the file + * @throws IOException if the IndexInput cannot be created or file does not exist + */ + public IndexInput openInput(FileMetadata fileMetadata, IOContext context) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + return formatDirectory.openIndexInput(fileMetadata.file(), context); + } + + /** + * Creates an IndexOutput for writing using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @param context IOContext providing performance hints for the operation + * @return IndexOutput for writing to the file + * @throws IOException if the IndexOutput cannot be created + */ + public IndexOutput createOutput(FileMetadata fileMetadata, IOContext context) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + OutputStream outputStream = formatDirectory.createOutput(fileMetadata.file()); + return new OutputStreamIndexOutput(outputStream, fileMetadata.file()); + } + + /** + * Copies a file from another directory using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @param source the source Directory to copy from + * @param context IOContext providing performance hints for the operation + * @throws IOException if the copy operation fails + */ + public void copyFrom(FileMetadata fileMetadata, RemoteSegmentStoreDirectory source, IOContext context) throws IOException { + FormatStoreDirectory targetDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + String fileName = fileMetadata.file(); + + logger.debug("Copying file {} to format directory: {}", fileName, targetDirectory.getDataFormat().name()); + + try (IndexInput input = source.openInput(fileMetadata, context); + IndexOutput output = createOutput(fileMetadata, context)) { + + output.copyBytes(input, input.length()); + } + } + + /** + * Calculates checksum using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @return the checksum as a long value + * @throws IOException if checksum calculation fails + */ + public long calculateChecksum(FileMetadata fileMetadata) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + return formatDirectory.calculateChecksum(fileMetadata.file()); + } + + /** + * Calculates upload checksum using FileMetadata for format routing + * @param fileMetadata the FileMetadata containing format and filename information + * @return checksum string in format-specific representation + * @throws IOException if checksum calculation fails + */ + public String calculateUploadChecksum(FileMetadata fileMetadata) throws IOException { + FormatStoreDirectory formatDirectory = getDirectoryForFormat(fileMetadata.dataFormat()); + return formatDirectory.calculateUploadChecksum(fileMetadata.file()); + } +} + +/** + * Adapter class that converts OutputStream to Lucene IndexOutput + */ +class OutputStreamIndexOutput extends IndexOutput { + private final OutputStream outputStream; + private final String name; + private long bytesWritten = 0; + + OutputStreamIndexOutput(OutputStream outputStream, String name) { + super("OutputStreamIndexOutput(" + name + ")", name); + this.outputStream = outputStream; + this.name = name; + } + + @Override + public void writeByte(byte b) throws IOException { + outputStream.write(b & 0xFF); + bytesWritten++; + } + + @Override + public void writeBytes(byte[] b, int offset, int length) throws IOException { + outputStream.write(b, offset, length); + bytesWritten += length; + } + + @Override + public long getFilePointer() { + return bytesWritten; + } + + @Override + public long getChecksum() throws IOException { + // Generic implementation - checksum not supported + throw new UnsupportedOperationException("Checksum not supported for generic output streams"); + } + + @Override + public void close() throws IOException { + outputStream.close(); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectoryFactory.java new file mode 100644 index 0000000000000..ad062573a8aa1 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CompositeStoreDirectoryFactory.java @@ -0,0 +1,49 @@ +/* + * 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.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.plugins.PluginsService; + +import java.io.IOException; + +/** + * Factory interface for creating CompositeStoreDirectory instances. + * This interface follows the existing IndexStorePlugin pattern to provide + * a centralized way to create composite directories with format discovery. + * + * @opensearch.experimental + */ +@ExperimentalApi +@FunctionalInterface +public interface CompositeStoreDirectoryFactory { + + /** + * Creates a new CompositeStoreDirectory per shard with automatic format discovery. + * + * The factory will: + * - Use PluginsService to discover available DataFormat plugins + * - Create format-specific directories for each discovered format + * - Provide fallback behavior if no plugins are found + * - Handle errors gracefully with proper logging + * + * @param indexSettings the shard's index settings containing configuration + * @param shardPath the path the shard is using for file storage + * @param pluginsService service for discovering DataFormat plugins and creating format directories + * @return a new CompositeStoreDirectory instance supporting all discovered formats + * @throws IOException if directory creation fails or resources cannot be allocated + */ + CompositeStoreDirectory newCompositeStoreDirectory( + IndexSettings indexSettings, + ShardPath shardPath, + PluginsService pluginsService + ) throws IOException; +} diff --git a/server/src/main/java/org/opensearch/index/store/DefaultCompositeStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/DefaultCompositeStoreDirectoryFactory.java new file mode 100644 index 0000000000000..be9689d07a7c2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/DefaultCompositeStoreDirectoryFactory.java @@ -0,0 +1,94 @@ +/* + * 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.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.coord.Any; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.plugins.PluginsService; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.logger; + +/** + * Default implementation of CompositeStoreDirectoryFactory that provides + * plugin-based format discovery and fallback behavior. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class DefaultCompositeStoreDirectoryFactory implements CompositeStoreDirectoryFactory { + + private static final Logger logger = LogManager.getLogger(DefaultCompositeStoreDirectoryFactory.class); + + /** + * Default formats used when no plugins are discovered. + * Includes Lucene (primary format) and Text (fallback format). + */ + private static final List DEFAULT_FORMATS = Arrays.asList( + DataFormat.LUCENE + ); + + /** + * Creates a new CompositeStoreDirectory with plugin-based format discovery. + * + * @param indexSettings the shard's index settings + * @param shardPath the path the shard is using + * @param pluginsService service for discovering DataFormat plugins + * @return a new CompositeStoreDirectory instance + * @throws IOException if directory creation fails + */ + @Override + public CompositeStoreDirectory newCompositeStoreDirectory( + IndexSettings indexSettings, + ShardPath shardPath, + PluginsService pluginsService + ) throws IOException { + + if (logger.isDebugEnabled()) { + logger.debug("Creating CompositeStoreDirectory for shard: {} at path: {}", + shardPath.getShardId(), shardPath.getDataPath()); + } + + try { + CompositeStoreDirectory compositeDirectory = new CompositeStoreDirectory( + indexSettings, + pluginsService, + shardPath, + logger + ); + + if (logger.isDebugEnabled()) { + logger.debug("Successfully created CompositeStoreDirectory for shard: {} with plugin discovery", + shardPath.getShardId()); + } + + return compositeDirectory; + + }catch (Exception fallbackException) { + logger.error("Failed to create CompositeStoreDirectory for shard: {} - both plugin discovery and fallback failed", + shardPath.getShardId(), fallbackException); + + throw new IOException( + String.format("Failed to create CompositeStoreDirectory for shard %s: plugin discovery failed (%s), fallback failed (%s)", + shardPath.getShardId(), + fallbackException.getMessage() + ), + fallbackException + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/FormatStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/FormatStoreDirectory.java new file mode 100644 index 0000000000000..ff1e4636e57b3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/FormatStoreDirectory.java @@ -0,0 +1,145 @@ +/* + * 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.IOContext; +import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.FileMetadata; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Path; +import java.util.Collection; + +/** + * Interface for format-specific store directories that handle file operations + * for different data formats (Lucene, Parquet, etc.) + * This interface is format-agnostic and does not depend on Lucene's Directory class. + * + * @opensearch.api + */ +@PublicApi(since = "3.0.0") +public interface FormatStoreDirectory extends Closeable { + + /** + * Returns the data format this directory handles + * @return the data format for this directory + */ + T getDataFormat(); + + /** + * Returns the directory path for this format + * @return the path to this format's directory + */ + Path getDirectoryPath(); + + /** + * Performs format-specific initialization + * @throws IOException if initialization fails + */ + void initialize() throws IOException; + + /** + * Performs format-specific cleanup + * @throws IOException if cleanup fails + */ + void cleanup() throws IOException; + + // Format-agnostic file operations using standard Java I/O + + /** + * Lists all files in this directory + * @return array of file names in the directory + * @throws IOException if listing fails + */ + FileMetadata[] listAll() throws IOException; + + /** + * Deletes the specified file + * @param name the name of the file to delete + * @throws IOException if deletion fails + */ + void deleteFile(String name) throws IOException; + + /** + * Returns the length of the specified file + * @param name the name of the file + * @return the length of the file in bytes + * @throws IOException if the file cannot be accessed + */ + long fileLength(String name) throws IOException; + + /** + * Creates an output stream for writing to the specified file + * @param name the name of the file to create + * @return an OutputStream for writing to the file + * @throws IOException if the output stream cannot be created + */ + OutputStream createOutput(String name) throws IOException; + + /** + * Opens an input stream for reading from the specified file + * @param name the name of the file to read + * @return an InputStream for reading from the file + * @throws IOException if the input stream cannot be opened + */ + InputStream openInput(String name) throws IOException; + + /** + * Syncs the specified files to persistent storage + * @param names collection of file names to sync + * @throws IOException if syncing fails + */ + void sync(Collection names) throws IOException; + + /** + * Renames a file + * @param source the current name of the file + * @param dest the new name for the file + * @throws IOException if renaming fails + */ + void rename(String source, String dest) throws IOException; + + /** + * Calculates the checksum for the specified file using format-specific method + * @param fileName the name of the file to calculate checksum for + * @return the checksum as a string representation + * @throws IOException if checksum calculation fails + */ + long calculateChecksum(String fileName) throws IOException; + + /** + * Calculates format-specific checksum for upload verification. + * This method provides format-appropriate checksum calculation that may differ + * from the general calculateChecksum method. For example, Lucene files use + * checksum-of-checksum calculation, while other formats may use SHA-256. + * + * @param fileName the name of the file to calculate checksum for + * @return checksum string in format-specific representation suitable for upload verification + * @throws IOException if checksum calculation fails or the file cannot be accessed + */ + String calculateUploadChecksum(String fileName) throws IOException; + + /** + * Opens an IndexInput for reading from the specified file. + * This method mirrors Lucene Directory.openInput() behavior, providing + * full random access, seeking, and cloning capabilities for compatibility + * with existing Lucene-based code. + * + * @param name the name of the file to read + * @param context IOContext providing performance hints for the operation + * @return IndexInput for reading from the file with full Lucene compatibility + * @throws IOException if the IndexInput cannot be created or file does not exist + */ + IndexInput openIndexInput(String name, IOContext context) throws IOException; +} diff --git a/server/src/main/java/org/opensearch/index/store/GenericStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/GenericStoreDirectory.java new file mode 100644 index 0000000000000..977fcfc4448ce --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/GenericStoreDirectory.java @@ -0,0 +1,719 @@ +/* + * 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.logging.log4j.Logger; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.shard.ShardPath; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.nio.file.StandardOpenOption; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Collection; +import java.util.Set; +import java.util.stream.StreamSupport; + +/** + * Generic FormatStoreDirectory implementation for non-Lucene formats. + * Uses core Java APIs to provide directory functionality without Lucene dependencies. + */ +public class GenericStoreDirectory implements FormatStoreDirectory { + + private final T dataFormat; + private final Path directoryPath; + private final Logger logger; + + /** + * Creates a new GenericStoreDirectory + * @param dataFormat the data format this directory handles + * @param shardPath the shard path where directories should be created + * @param logger logger for this directory + * @throws IOException if directory creation fails + */ + public GenericStoreDirectory( + T dataFormat, + Path shardPath, + Logger logger + ) throws IOException { + this.dataFormat = dataFormat; + this.directoryPath = shardPath.resolve(dataFormat.name()); + this.logger = logger; + + Files.createDirectories(this.directoryPath); + } + + + @Override + public T getDataFormat() { + return dataFormat; + } + + @Override + public Path getDirectoryPath() { + return directoryPath; + } + + @Override + public void initialize() throws IOException { + logger.debug("Initialized GenericStoreDirectory for format: {}", dataFormat.name()); + } + + @Override + public void cleanup() throws IOException { + logger.debug("Cleaning up GenericStoreDirectory for format: {}", dataFormat.name()); + } + + @Override + public FileMetadata[] listAll() throws IOException { + try (DirectoryStream stream = Files.newDirectoryStream(directoryPath)) { + return StreamSupport.stream(stream.spliterator(), false) + .map(Path::getFileName) + .map(Path::toString) + .filter(name -> !Files.isDirectory(directoryPath.resolve(name))) + .map(fileName -> new FileMetadata(this.dataFormat.name(), "", fileName)) // Create FileMetadata with format + filename + .toArray(FileMetadata[]::new); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to list files in directory", + dataFormat, + "listAll", + directoryPath, + e + ); + } + } + + @Override + public void deleteFile(String name) throws IOException { + Path filePath = directoryPath.resolve(name); + try { + Files.deleteIfExists(filePath); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to delete file: " + name, + dataFormat, + "deleteFile", + filePath, + e + ); + } + } + + @Override + public long fileLength(String name) throws IOException { + Path filePath = directoryPath.resolve(name); + try { + return Files.size(filePath); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to get file length: " + name, + dataFormat, + "fileLength", + filePath, + e + ); + } + } + + @Override + public OutputStream createOutput(String name) throws IOException { + Path filePath = directoryPath.resolve(name); + try { + return Files.newOutputStream(filePath, + StandardOpenOption.CREATE, + StandardOpenOption.WRITE, + StandardOpenOption.TRUNCATE_EXISTING); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to create output stream for file: " + name, + dataFormat, + "createOutput", + filePath, + e + ); + } + } + + @Override + public InputStream openInput(String name) throws IOException { + Path filePath = directoryPath.resolve(name); + try { + return Files.newInputStream(filePath, StandardOpenOption.READ); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to open input stream for file: " + name, + dataFormat, + "openInput", + filePath, + e + ); + } + } + + @Override + public void sync(Collection names) throws IOException { + // For generic directories, sync each file by calling fsync + for (String name : names) { + Path filePath = directoryPath.resolve(name); + if (Files.exists(filePath)) { + try (FileChannel channel = FileChannel.open(filePath, StandardOpenOption.WRITE)) { + channel.force(true); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to sync file: " + name, + dataFormat, + "sync", + filePath, + e + ); + } + } + } + } + + @Override + public void rename(String source, String dest) throws IOException { + Path sourcePath = directoryPath.resolve(source); + Path destPath = directoryPath.resolve(dest); + try { + Files.move(sourcePath, destPath, StandardCopyOption.REPLACE_EXISTING); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to rename file from " + source + " to " + dest, + dataFormat, + "rename", + sourcePath, + e + ); + } + } + + @Override + public long calculateChecksum(String fileName) throws IOException { + Path filePath = directoryPath.resolve(fileName); + try (InputStream inputStream = Files.newInputStream(filePath, StandardOpenOption.READ)) { + return calculateGenericChecksum(inputStream); + } catch (IOException e) { + throw new MultiFormatStoreException( + "Failed to calculate checksum for file: " + fileName, + dataFormat, + "calculateChecksum", + filePath, + e + ); + } + } + + /** + * Calculates a generic CRC32 checksum for the given input stream + * @param inputStream the input stream to calculate checksum for + * @return the checksum as a string representation + * @throws IOException if reading from the stream fails + */ + private long calculateGenericChecksum(InputStream inputStream) throws IOException { + java.util.zip.CRC32 crc32 = new java.util.zip.CRC32(); + byte[] buffer = new byte[8192]; + int bytesRead; + + while ((bytesRead = inputStream.read(buffer)) != -1) { + crc32.update(buffer, 0, bytesRead); + } + + return crc32.getValue(); + } + + @Override + public void close() throws IOException { + // No resources to close for generic directory + logger.debug("Closed GenericStoreDirectory for format: {}", dataFormat.name()); + } + + @Override + public String calculateUploadChecksum(String fileName) throws IOException { + if (fileName == null || fileName.trim().isEmpty()) { + throw new IllegalArgumentException("File name cannot be null or empty"); + } + + Path filePath = directoryPath.resolve(fileName); + + logger.debug("Calculating generic upload checksum: file={}, format={}, method=CRC32, filePath={}", + fileName, dataFormat.name(), filePath); + + long startTime = System.nanoTime(); + + try (InputStream inputStream = Files.newInputStream(filePath)) { + long checksum = calculateGenericChecksum(inputStream); + String checksumString = Long.toString(checksum); + + long calculationDurationMs = (System.nanoTime() - startTime) / 1_000_000; + + logger.debug("Generic upload checksum calculated: file={}, format={}, checksum={}, durationMs={}", + fileName, dataFormat.name(), checksumString, calculationDurationMs); + + return checksumString; + + } catch (IOException e) { + long failureDurationMs = (System.nanoTime() - startTime) / 1_000_000; + + logger.error("Failed to calculate generic upload checksum: file={}, format={}, filePath={}, durationMs={}, error={}", + fileName, dataFormat.name(), filePath, failureDurationMs, e.getMessage(), e); + + throw new MultiFormatStoreException( + "Failed to calculate upload checksum for file: " + fileName, + dataFormat, + "calculateUploadChecksum", + filePath, + e + ); + } + } + + @Override + public IndexInput openIndexInput(String name, IOContext context) throws IOException { + if (name == null || name.trim().isEmpty()) { + throw new IllegalArgumentException("File name cannot be null or empty"); + } + + Path filePath = directoryPath.resolve(name); + + logger.debug("Creating IndexInput for generic format: file={}, format={}, context={}, filePath={}", + name, dataFormat.name(), context, filePath); + + try { + // Validate file exists + if (!Files.exists(filePath) || !Files.isRegularFile(filePath)) { + throw new IOException("File does not exist or is not a regular file: " + filePath); + } + + // Open FileChannel + FileChannel channel = FileChannel.open(filePath, StandardOpenOption.READ); + long fileSize = channel.size(); + + // Create FileChannel-based IndexInput + return new GenericFileChannelIndexInput(name, channel, fileSize, context); + + } catch (IOException e) { + logger.error("Failed to create IndexInput for generic format: file={}, format={}, filePath={}, error={}", + name, dataFormat.name(), filePath, e.getMessage(), e); + + throw new MultiFormatStoreException( + "Failed to create IndexInput for file: " + name, + dataFormat, + "openIndexInput", + filePath, + e + ); + } + } + + /** + * FileChannel-based IndexInput implementation that provides full Lucene compatibility. + * This implementation mirrors NIOFSDirectory's internal IndexInput behavior. + */ + private static class GenericFileChannelIndexInput extends IndexInput implements RandomAccessInput { + private final FileChannel channel; + private final long length; + private final IOContext context; + private final boolean isClone; + private long position = 0; + private volatile boolean closed = false; + + GenericFileChannelIndexInput(String name, FileChannel channel, long length, IOContext context) { + this(name, channel, length, context, false); + } + + private GenericFileChannelIndexInput(String name, FileChannel channel, long length, IOContext context, boolean isClone) { + super("GenericFileChannelIndexInput(" + name + ")"); + this.channel = channel; + this.length = length; + this.context = context; + this.isClone = isClone; + } + + + @Override + public byte readByte() throws IOException { + checkClosed(); + if (position >= length) { + throw new EOFException("Read past EOF: position=" + position + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(1); + int bytesRead = channel.read(buffer, position); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + position); + } + + position++; + return buffer.get(0); + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException { + checkClosed(); + if (position + len > length) { + throw new EOFException("Read past EOF: position=" + position + + ", requestedLen=" + len + ", fileLength=" + length); + } + + ByteBuffer buffer = ByteBuffer.wrap(b, offset, len); + int totalRead = 0; + while (totalRead < len) { + int bytesRead = channel.read(buffer, position + totalRead); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + (position + totalRead)); + } + totalRead += bytesRead; + } + position += len; + } + + @Override + public long getFilePointer() { + return position; + } + + @Override + public void seek(long pos) throws IOException { + checkClosed(); + if (pos < 0) { + throw new IllegalArgumentException("Seek position cannot be negative: " + pos); + } + if (pos > length) { + throw new EOFException("Seek past EOF: position=" + pos + ", length=" + length); + } + + position = pos; + } + + @Override + public long length() { + return length; + } + + + @Override + public byte readByte(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos >= length) { + throw new EOFException("Position out of bounds: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(1); + int bytesRead = channel.read(buffer, pos); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + return buffer.get(0); + } + + @Override + public short readShort(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Short.BYTES > length) { + throw new EOFException("Position out of bounds: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Short.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, pos); + if (bytesRead != Short.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getShort(); // ✅ Multi-byte random access + } + + @Override + public int readInt(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Integer.BYTES > length) { + throw new EOFException("Position out of bounds: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, pos); + if (bytesRead != Integer.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getInt(); // ✅ Multi-byte random access + } + + @Override + public long readLong(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Long.BYTES > length) { + throw new EOFException("Position out of bounds: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, pos); + if (bytesRead != Long.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getLong(); + } + + + @Override + public GenericFileChannelIndexInput clone() { + checkClosed(); + try { + // Create new FileChannel for the same file (like Lucene does) + // Unfortunately, we can't get the original path from FileChannel easily, + // so we'll share the same channel for now (requires careful resource management) + GenericFileChannelIndexInput clone = new GenericFileChannelIndexInput( + toString(), channel, length, context, true + ); + clone.position = this.position; + + return clone; + + } catch (Exception e) { + throw new RuntimeException("Failed to clone IndexInput", e); + } + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long sliceLength) throws IOException { + checkClosed(); + if (offset < 0 || sliceLength < 0 || offset + sliceLength > length) { + throw new IllegalArgumentException( + "Invalid slice: offset=" + offset + ", length=" + sliceLength + + ", fileLength=" + length + ); + } + + // Create slice that shares the same FileChannel but with offset/length bounds + return new SlicedGenericIndexInput( + sliceDescription, channel, offset, sliceLength, context + ); + } + + @Override + public void close() throws IOException { + if (!closed && !isClone) { // Only close original, not clones + channel.close(); + closed = true; + } + } + + private void checkClosed() { + if (closed) { + throw new RuntimeException("IndexInput is closed"); + } + } + } + + /** + * Sliced version of GenericFileChannelIndexInput that operates within bounds + */ + private static class SlicedGenericIndexInput extends IndexInput implements RandomAccessInput { + private final FileChannel channel; + private final long startOffset; + private final long length; + private final IOContext context; + private long position = 0; + private volatile boolean closed = false; + + SlicedGenericIndexInput(String name, FileChannel channel, long startOffset, long length, IOContext context) { + super("SlicedGenericIndexInput(" + name + ")"); + this.channel = channel; + this.startOffset = startOffset; + this.length = length; + this.context = context; + } + + @Override + public byte readByte() throws IOException { + checkClosed(); + if (position >= length) { + throw new EOFException("Read past EOF in slice: position=" + position + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(1); + int bytesRead = channel.read(buffer, startOffset + position); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + position); + } + + position++; + return buffer.get(0); + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException { + checkClosed(); + if (position + len > length) { + throw new EOFException("Read past EOF in slice: position=" + position + + ", requestedLen=" + len + ", sliceLength=" + length); + } + + ByteBuffer buffer = ByteBuffer.wrap(b, offset, len); + int totalRead = 0; + while (totalRead < len) { + int bytesRead = channel.read(buffer, startOffset + position + totalRead); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + (position + totalRead)); + } + totalRead += bytesRead; + } + position += len; + } + + @Override + public long getFilePointer() { + return position; + } + + @Override + public void seek(long pos) throws IOException { + checkClosed(); + if (pos < 0) { + throw new IllegalArgumentException("Seek position cannot be negative: " + pos); + } + if (pos > length) { + throw new EOFException("Seek past EOF in slice: position=" + pos + ", length=" + length); + } + + position = pos; + } + + @Override + public long length() { + return length; + } + + @Override + public byte readByte(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos >= length) { + throw new EOFException("Position out of bounds in slice: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(1); + int bytesRead = channel.read(buffer, startOffset + pos); + if (bytesRead == -1) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + return buffer.get(0); + } + + @Override + public short readShort(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Short.BYTES > length) { + throw new EOFException("Position out of bounds in slice: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Short.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, startOffset + pos); + if (bytesRead != Short.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getShort(); + } + + @Override + public int readInt(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Integer.BYTES > length) { + throw new EOFException("Position out of bounds in slice: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, startOffset + pos); + if (bytesRead != Integer.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getInt(); + } + + @Override + public long readLong(long pos) throws IOException { + checkClosed(); + if (pos < 0 || pos + Long.BYTES > length) { + throw new EOFException("Position out of bounds in slice: pos=" + pos + ", length=" + length); + } + + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES).order(java.nio.ByteOrder.LITTLE_ENDIAN); + int bytesRead = channel.read(buffer, startOffset + pos); + if (bytesRead != Long.BYTES) { + throw new EOFException("Unexpected EOF at position: " + pos); + } + + buffer.flip(); + return buffer.getLong(); + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long sliceLength) throws IOException { + checkClosed(); + if (offset < 0 || sliceLength < 0 || offset + sliceLength > length) { + throw new IllegalArgumentException( + "Invalid slice: offset=" + offset + ", length=" + sliceLength + + ", parentLength=" + length + ); + } + + return new SlicedGenericIndexInput( + sliceDescription, channel, startOffset + offset, sliceLength, context + ); + } + + @Override + public SlicedGenericIndexInput clone() { + checkClosed(); + SlicedGenericIndexInput clone = new SlicedGenericIndexInput( + toString(), channel, startOffset, length, context + ); + clone.position = this.position; + return clone; + } + + @Override + public void close() throws IOException { + if (!closed) { + // Slices don't own the channel, so don't close it + closed = true; + } + } + + private void checkClosed() { + if (closed) { + throw new RuntimeException("IndexInput is closed"); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/LuceneStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/LuceneStoreDirectory.java new file mode 100644 index 0000000000000..eff7bacb7219c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/LuceneStoreDirectory.java @@ -0,0 +1,205 @@ +/* + * 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.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.FileMetadata; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Set; + +/** + * FormatStoreDirectory implementation for Lucene format files. + * Wraps existing Lucene Directory to maintain full Lucene compatibility. + * It will be used when Lucene format will be supported as well. + */ +public class LuceneStoreDirectory implements FormatStoreDirectory { + + private static final Logger logger = LogManager.getLogger(LuceneStoreDirectory.class); + + private final Directory wrappedDirectory; + private final Path directoryPath; + + public LuceneStoreDirectory( + Path shardPath, + Directory directory + ) throws IOException { + this.directoryPath = shardPath.resolve("lucene"); + Files.createDirectories(this.directoryPath); + + // Use the provided Directory directly + this.wrappedDirectory = directory; + } + + @Override + public DataFormat getDataFormat() { + return DataFormat.LUCENE; + } + + @Override + public Path getDirectoryPath() { + return directoryPath; + } + + @Override + public void initialize() throws IOException { + // Lucene-specific initialization if needed + } + + @Override + public void cleanup() throws IOException { + // Lucene-specific cleanup if needed + } + + // Implement FormatStoreDirectory methods by delegating to wrappedDirectory + @Override + public FileMetadata[] listAll() throws IOException { + String[] fileNames = wrappedDirectory.listAll(); + FileMetadata[] fileMetadataArray = new FileMetadata[fileNames.length]; + + String dataFormat = getDataFormat().toString(); // "LUCENE" + + for (int i = 0; i < fileNames.length; i++) { + fileMetadataArray[i] = new FileMetadata(dataFormat, "", fileNames[i]); + } + + return fileMetadataArray; + } + + @Override + public void deleteFile(String name) throws IOException { + wrappedDirectory.deleteFile(name); + } + + @Override + public long fileLength(String name) throws IOException { + return wrappedDirectory.fileLength(name); + } + + @Override + public OutputStream createOutput(String name) throws IOException { + // Convert Lucene IndexOutput to OutputStream + IndexOutput indexOutput = wrappedDirectory.createOutput(name, IOContext.DEFAULT); + return new OutputStream() { + @Override + public void write(int b) throws IOException { + indexOutput.writeByte((byte) b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + indexOutput.writeBytes(b, off, len); + } + + @Override + public void close() throws IOException { + indexOutput.close(); + } + }; + } + + @Override + public InputStream openInput(String name) throws IOException { + // Convert Lucene IndexInput to InputStream + IndexInput indexInput = wrappedDirectory.openInput(name, IOContext.DEFAULT); + return new InputStream() { + @Override + public int read() throws IOException { + if (indexInput.getFilePointer() >= indexInput.length()) { + return -1; + } + return indexInput.readByte() & 0xFF; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + long remaining = indexInput.length() - indexInput.getFilePointer(); + if (remaining <= 0) { + return -1; + } + + int toRead = (int) Math.min(len, remaining); + indexInput.readBytes(b, off, toRead); + return toRead; + } + + @Override + public void close() throws IOException { + indexInput.close(); + } + }; + } + + @Override + public void sync(Collection names) throws IOException { + wrappedDirectory.sync(names); + } + + @Override + public void rename(String source, String dest) throws IOException { + wrappedDirectory.rename(source, dest); + } + + @Override + public void close() throws IOException { + wrappedDirectory.close(); + } + + @Override + public long calculateChecksum(String fileName) throws IOException { + try (IndexInput indexInput = wrappedDirectory.openInput(fileName, IOContext.READONCE)) { + return CodecUtil.retrieveChecksum(indexInput); + } + } + + @Override + public String calculateUploadChecksum(String fileName) throws IOException { + if (fileName == null || fileName.trim().isEmpty()) { + throw new IllegalArgumentException("File name cannot be null or empty"); + } + + logger.debug("Calculating Lucene upload checksum: file={}, method=checksum-of-checksum", fileName); + + long startTime = System.nanoTime(); + + // Use existing Lucene checksum calculation logic + String checksum; + try (IndexInput indexInput = wrappedDirectory.openInput(fileName, IOContext.READONCE)) { + checksum = Long.toString(org.apache.lucene.codecs.CodecUtil.retrieveChecksum(indexInput)); + } catch (Exception e) { + throw new IOException("Failed to calculate Lucene checksum for " + fileName, e); + } + + long calculationDurationMs = (System.nanoTime() - startTime) / 1_000_000; + + logger.debug("Lucene upload checksum calculated: file={}, checksum={}, durationMs={}", + fileName, checksum, calculationDurationMs); + + return checksum; + } + + @Override + public IndexInput openIndexInput(String name, IOContext context) throws IOException { + return wrappedDirectory.openInput(name, context); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/MultiFormatStoreException.java b/server/src/main/java/org/opensearch/index/store/MultiFormatStoreException.java new file mode 100644 index 0000000000000..a5be60485bc00 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/MultiFormatStoreException.java @@ -0,0 +1,98 @@ +/* + * 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.opensearch.index.engine.exec.DataFormat; + +import java.io.IOException; +import java.nio.file.Path; + +/** + * Exception for multi-format store operations that provides comprehensive + * error context including format, operation, and path information. + */ +public class MultiFormatStoreException extends IOException { + + private final DataFormat affectedFormat; + private final String operation; + private final Path attemptedPath; + + /** + * Creates a new MultiFormatStoreException with comprehensive context + * + * @param message the error message + * @param affectedFormat the data format that was being operated on + * @param operation the operation that failed + * @param attemptedPath the path that was being accessed + * @param cause the underlying cause of the exception + */ + public MultiFormatStoreException( + String message, + DataFormat affectedFormat, + String operation, + Path attemptedPath, + Throwable cause + ) { + super(formatMessage(message, affectedFormat, operation, attemptedPath), cause); + this.affectedFormat = affectedFormat; + this.operation = operation; + this.attemptedPath = attemptedPath; + } + + /** + * Formats the error message with comprehensive context information + * + * @param message the base error message + * @param format the affected data format + * @param operation the failed operation + * @param path the attempted path + * @return formatted error message with context + */ + private static String formatMessage( + String message, + DataFormat format, + String operation, + Path path + ) { + return String.format( + "Multi-format store operation failed: %s [format=%s, operation=%s, path=%s]", + message, + format != null ? format.name() : "unknown", + operation != null ? operation : "unknown", + path != null ? path.toString() : "unknown" + ); + } + + /** + * Returns the data format that was affected by this exception + * + * @return the affected data format, or null if unknown + */ + public DataFormat getAffectedFormat() { + return affectedFormat; + } + + /** + * Returns the operation that failed + * + * @return the failed operation name, or null if unknown + */ + public String getOperation() { + return operation; + } + + /** + * Returns the path that was being accessed when the exception occurred + * + * @return the attempted path, or null if unknown + */ + public Path getAttemptedPath() { + return attemptedPath; + } +} diff --git a/server/src/main/java/org/opensearch/index/store/ParquetDataFormatPlugin.java b/server/src/main/java/org/opensearch/index/store/ParquetDataFormatPlugin.java new file mode 100644 index 0000000000000..2e8c8e517b7c5 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/ParquetDataFormatPlugin.java @@ -0,0 +1,150 @@ +/* + * 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.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.DataFormatPlugin; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.IndexingExecutionEngine; +import org.opensearch.index.shard.ShardPath; + +import java.io.IOException; +import java.util.Set; +// +///** +// * DataFormatPlugin implementation for Parquet format support. +// * Provides Parquet-specific store directory creation and file extension handling. +// */ +//public class ParquetDataFormatPlugin implements DataFormatPlugin { +// +// /** +// * Singleton instance for the Parquet data format plugin +// */ +// public static final ParquetDataFormatPlugin INSTANCE = new ParquetDataFormatPlugin(); +// +// /** +// * Set of file extensions that Parquet format handles +// */ +// private static final Set PARQUET_EXTENSIONS = Set.of(".parquet", ".pqt"); +// +// /** +// * Private constructor to enforce singleton pattern +// */ +// private ParquetDataFormatPlugin() { +// } +// +// @Override +// public IndexingExecutionEngine indexingEngine() { +// // For now, return null as this is not implemented yet +// // This will be implemented when the indexing engine is needed +// return null; +// } +// +// @Override +// public DataFormat getDataFormat() { +// return new ParquetDataFormat(); +// } +// +// @Override +// public FormatStoreDirectory createFormatStoreDirectory( +// IndexSettings indexSettings, +// ShardPath shardPath +// ) throws IOException { +// // Create a GenericStoreDirectory for the parquet subdirectory +// Logger logger = LogManager.getLogger("index.store.parquet." + shardPath.getShardId()); +// +// return new GenericStoreDirectory( +// new ParquetDataFormat(), +// shardPath.getDataPath(), +// PARQUET_EXTENSIONS, +// logger +// ); +// } +// +// @Override +// public FormatRemoteDirectory createFormatRemoteDirectory( +// IndexSettings indexSettings, +// BlobContainer baseBlobContainer, +// String remoteBasePath +// ) throws IOException { +// // Create GenericRemoteDirectory for Parquet format +// ParquetDataFormat parquetFormat = new ParquetDataFormat(); +// Logger logger = LogManager.getLogger("index.store.remote.parquet"); +// return new GenericRemoteDirectory( +// parquetFormat, +// remoteBasePath, +// baseBlobContainer, +// PARQUET_EXTENSIONS, +// logger +// ); +// } +// +// @Override +// public FormatRemoteDirectory createFormatRemoteDirectory( +// IndexSettings indexSettings, +// RemoteDirectory remoteDirectory, +// String remoteBasePath +// ) throws IOException { +// // For Parquet format, we use BlobContainer-based approach +// // This method should not be called for Parquet format +// throw new UnsupportedOperationException( +// "Parquet format uses BlobContainer-based remote storage. Use createFormatRemoteDirectory(IndexSettings, BlobContainer, String) instead." +// ); +// } +// +// @Override +// public boolean supportsRemoteStorage() { +// return true; +// } +// +// @Override +// public String getRemotePathSuffix() { +// return "parquet"; +// } +// +// @Override +// public Set getSupportedExtensions() { +// return PARQUET_EXTENSIONS; +// } +// +// /** +// * Parquet DataFormat implementation +// */ +// public static class ParquetDataFormat implements DataFormat { +// @Override +// public Setting dataFormatSettings() { +// return null; +// } +// +// @Override +// public Setting clusterLeveldataFormatSettings() { +// return null; +// } +// +// @Override +// public String name() { +// return "parquet"; +// } +// +// @Override +// public void configureStore() { +// +// } +// +// @Override +// public String getDirectoryName() { +// return "parquet"; +// } +// } +//} diff --git a/server/src/main/java/org/opensearch/index/store/ParquetStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/ParquetStoreDirectory.java new file mode 100644 index 0000000000000..11d7673eec471 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/ParquetStoreDirectory.java @@ -0,0 +1,308 @@ +/* + * 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.logging.log4j.LogManager; +//import org.apache.logging.log4j.Logger; +//import org.apache.lucene.store.IOContext; +//import org.apache.lucene.store.IndexInput; +//import org.opensearch.index.engine.exec.DataFormat; +// +//import java.io.FileNotFoundException; +//import java.io.IOException; +//import java.io.InputStream; +//import java.io.OutputStream; +//import java.nio.file.Files; +//import java.nio.file.NoSuchFileException; +//import java.nio.file.Path; +//import java.util.Collection; +// +///** +// * FormatStoreDirectory implementation for Parquet format files. +// * Handles storage and retrieval of Parquet-formatted data files. +// */ +//public class ParquetStoreDirectory implements FormatStoreDirectory { +// +// private static final Logger logger = LogManager.getLogger(ParquetStoreDirectory.class); +// +// private final Path directoryPath; +// +// public ParquetStoreDirectory(Path shardPath) throws IOException { +// this.directoryPath = shardPath.resolve("parquet"); +// Files.createDirectories(this.directoryPath); +// } +// +// @Override +// public DataFormat getDataFormat() { +// return DataFormat.PARQUET; +// } +// +// @Override +// public boolean supportsFormat(DataFormat format) { +// return FormatStoreDirectory.super.supportsFormat(format); +// } +// +// @Override +// @Deprecated +// public boolean acceptsFile(String fileName) { +// // This method is deprecated - format determination should be done through FileMetadata +// // For backward compatibility, check for parquet file extensions +// return fileName.endsWith(".parquet") || fileName.endsWith(".pq"); +// } +// +// @Override +// public Path getDirectoryPath() { +// return directoryPath; +// } +// +// @Override +// public void initialize() throws IOException { +// // Parquet-specific initialization if needed +// logger.debug("Initializing ParquetStoreDirectory at path: {}", directoryPath); +// } +// +// @Override +// public void cleanup() throws IOException { +// // Parquet-specific cleanup if needed +// logger.debug("Cleaning up ParquetStoreDirectory at path: {}", directoryPath); +// } +// +// @Override +// public String[] listAll() throws IOException { +// if (!Files.exists(directoryPath)) { +// return new String[0]; +// } +// return Files.list(directoryPath) +// .map(path -> path.getFileName().toString()) +// .toArray(String[]::new); +// } +// +// @Override +// public void deleteFile(String name) throws IOException { +// Path filePath = directoryPath.resolve(name); +// if (!Files.deleteIfExists(filePath)) { +// throw new NoSuchFileException("File does not exist: " + name); +// } +// } +// +// @Override +// public long fileLength(String name) throws IOException { +// Path filePath = directoryPath.resolve(name); +// if (!Files.exists(filePath)) { +// throw new NoSuchFileException("File does not exist: " + name); +// } +// return Files.size(filePath); +// } +// +// @Override +// public OutputStream createOutput(String name) throws IOException { +// Path filePath = directoryPath.resolve(name); +// return Files.newOutputStream(filePath); +// } +// +// @Override +// public InputStream openInput(String name) throws IOException { +// Path filePath = directoryPath.resolve(name); +// if (!Files.exists(filePath)) { +// throw new FileNotFoundException("File does not exist: " + name); +// } +// return Files.newInputStream(filePath); +// } +// +// @Override +// public void sync(Collection names) throws IOException { +// // Parquet files are typically written atomically, so sync is usually not needed +// // But we can implement it for completeness +// for (String name : names) { +// Path filePath = directoryPath.resolve(name); +// if (Files.exists(filePath)) { +// // Force sync to disk - this is a no-op on most filesystems for regular files +// // but ensures data is written to storage +// } +// } +// } +// +// @Override +// public void syncMetaData() throws IOException { +// // Sync directory metadata +// // This is typically a no-op but ensures directory changes are persisted +// } +// +// @Override +// public void rename(String source, String dest) throws IOException { +// Path sourcePath = directoryPath.resolve(source); +// Path destPath = directoryPath.resolve(dest); +// +// if (!Files.exists(sourcePath)) { +// throw new NoSuchFileException("Source file does not exist: " + source); +// } +// +// Files.move(sourcePath, destPath); +// } +// +// @Override +// public boolean fileExists(String name) throws IOException { +// Path filePath = directoryPath.resolve(name); +// return Files.exists(filePath); +// } +// +// @Override +// public void close() throws IOException { +// // Nothing to close for file-based directory +// } +// +// @Override +// public long calculateChecksum(String fileName) throws IOException { +// // For Parquet files, we can calculate a simple checksum +// // This is a basic implementation - could be enhanced with CRC32 or other algorithms +// Path filePath = directoryPath.resolve(fileName); +// if (!Files.exists(filePath)) { +// throw new NoSuchFileException("File does not exist: " + fileName); +// } +// +// // Simple checksum based on file size and modification time +// long size = Files.size(filePath); +// long lastModified = Files.getLastModifiedTime(filePath).toMillis(); +// return size ^ lastModified; +// } +// +// @Override +// public InputStream createUploadInputStream(String fileName) throws IOException { +// if (fileName == null || fileName.trim().isEmpty()) { +// throw new IllegalArgumentException("File name cannot be null or empty"); +// } +// +// logger.debug("Creating Parquet upload input stream: file={}, directoryPath={}", +// fileName, directoryPath.resolve(fileName)); +// +// Path filePath = directoryPath.resolve(fileName); +// if (!Files.exists(filePath)) { +// throw new FileNotFoundException("Parquet file does not exist: " + fileName); +// } +// +// return Files.newInputStream(filePath); +// } +// +// @Override +// public InputStream createUploadRangeInputStream(String fileName, long offset, long length) throws IOException { +// if (fileName == null || fileName.trim().isEmpty()) { +// throw new IllegalArgumentException("File name cannot be null or empty"); +// } +// if (offset < 0) { +// throw new IllegalArgumentException("Offset cannot be negative: " + offset); +// } +// if (length <= 0) { +// throw new IllegalArgumentException("Length must be positive: " + length); +// } +// +// Path filePath = directoryPath.resolve(fileName); +// if (!Files.exists(filePath)) { +// throw new FileNotFoundException("Parquet file does not exist: " + fileName); +// } +// +// long fileSize = Files.size(filePath); +// if (offset >= fileSize) { +// throw new IllegalArgumentException("Offset " + offset + " is beyond file length " + fileSize); +// } +// if (offset + length > fileSize) { +// throw new IllegalArgumentException("Range [" + offset + ", " + (offset + length) + +// ") exceeds file length " + fileSize); +// } +// +// InputStream fullStream = Files.newInputStream(filePath); +// +// // Skip to the offset +// long skipped = fullStream.skip(offset); +// if (skipped != offset) { +// fullStream.close(); +// throw new IOException("Could not skip to offset " + offset + ", only skipped " + skipped); +// } +// +// // Return a limited stream that only reads the specified length +// return new InputStream() { +// private long remaining = length; +// +// @Override +// public int read() throws IOException { +// if (remaining <= 0) { +// return -1; +// } +// int result = fullStream.read(); +// if (result != -1) { +// remaining--; +// } +// return result; +// } +// +// @Override +// public int read(byte[] b, int off, int len) throws IOException { +// if (remaining <= 0) { +// return -1; +// } +// int toRead = (int) Math.min(len, remaining); +// int bytesRead = fullStream.read(b, off, toRead); +// if (bytesRead > 0) { +// remaining -= bytesRead; +// } +// return bytesRead; +// } +// +// @Override +// public void close() throws IOException { +// fullStream.close(); +// } +// }; +// } +// +// @Override +// public String calculateUploadChecksum(String fileName) throws IOException { +// if (fileName == null || fileName.trim().isEmpty()) { +// throw new IllegalArgumentException("File name cannot be null or empty"); +// } +// +// logger.debug("Calculating Parquet upload checksum: file={}", fileName); +// +// long startTime = System.nanoTime(); +// +// // Calculate checksum for Parquet file +// long checksum = calculateChecksum(fileName); +// String checksumStr = Long.toString(checksum); +// +// long calculationDurationMs = (System.nanoTime() - startTime) / 1_000_000; +// +// logger.debug("Parquet upload checksum calculated: file={}, checksum={}, durationMs={}", +// fileName, checksumStr, calculationDurationMs); +// +// return checksumStr; +// } +// +// @Override +// public void onUploadComplete(String fileName, String remoteFileName) throws IOException { +// if (fileName == null || fileName.trim().isEmpty()) { +// throw new IllegalArgumentException("File name cannot be null or empty"); +// } +// if (remoteFileName == null || remoteFileName.trim().isEmpty()) { +// throw new IllegalArgumentException("Remote file name cannot be null or empty"); +// } +// +// logger.debug("Parquet upload completed: localFile={}, remoteFile={}, format={}, directoryPath={}", +// fileName, remoteFileName, getDataFormat().name(), directoryPath.resolve(fileName)); +// +// // Future enhancements could include: +// // - Updating local metadata about uploaded Parquet files +// // - Triggering Parquet-specific cleanup operations +// // - Notifying Parquet-specific monitoring systems +// // - Validating Parquet file integrity post-upload +// } +// +// @Override +// public IndexInput openIndexInput(String name, IOContext context) throws IOException { +// return null; +// } +//} 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 25ee020de8562..eb65192c5f140 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -384,7 +384,6 @@ public boolean copyFrom( } return false; } - private void uploadBlob( Directory from, String src, 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 af8382e2a3154..4829efd895a15 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfo; @@ -34,23 +33,27 @@ import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; +import org.opensearch.index.store.remote.CompositeRemoteDirectory; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandlerFactory; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.threadpool.ThreadPool; +import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.nio.file.NoSuchFileException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -63,16 +66,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; /** - * A RemoteDirectory extension for remote segment store. We need to make sure we don't overwrite a segment file once uploaded. - * In order to prevent segment overwrite which can occur due to two primary nodes for the same shard at the same time, - * a unique suffix is added to the uploaded segment file. This class keeps track of filename of segments stored - * in remote segment store vs filename in local filesystem and provides the consistent Directory interface so that - * caller will be accessing segment files in the same way as {@code FSDirectory}. Apart from storing actual segment files, - * remote segment store also keeps track of refresh checkpoints as metadata in a separate path which is handled by - * another instance of {@code RemoteDirectory}. + * Remote segment store directory with format-aware storage capabilities. + * Uses CompositeRemoteDirectory for all file operations with format-specific routing. * * @opensearch.api */ @@ -86,32 +83,33 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement public static final String SEGMENT_NAME_UUID_SEPARATOR = "__"; /** - * remoteDataDirectory is used to store segment files at path: cluster_UUID/index_UUID/shardId/segments/data + * compositeRemoteDirectory is used to store segment files with format-specific routing + * Always present - never null */ - private final RemoteDirectory remoteDataDirectory; + private final CompositeRemoteDirectory compositeRemoteDirectory; + RemoteDirectory remoteDataDirectory; + /** * remoteMetadataDirectory is used to store metadata files at path: cluster_UUID/index_UUID/shardId/segments/metadata */ private final RemoteDirectory remoteMetadataDirectory; private final RemoteStoreLockManager mdLockManager; - private final Map metadataFilePinnedTimestampMap; - private final ThreadPool threadPool; /** - Only relevant for remote-store-enabled domains on replica shards - to store localSegmentFilename -> remoteSegmentFilename mappings + * Only relevant for remote-store-enabled domains on replica shards + * to store localSegmentFilename -> remoteSegmentFilename mappings */ - private final Map pendingDownloadMergedSegments; + private final Map pendingDownloadMergedSegments; /** * Keeps track of local segment filename to uploaded filename along with other attributes like checksum. * This map acts as a cache layer for uploaded segment filenames which helps avoid calling listAll() each time. * It is important to initialize this map on creation of RemoteSegmentStoreDirectory and update it on each upload and delete. */ - private Map segmentsUploadedToRemoteStore; + private Map segmentsUploadedToRemoteStore; private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( new RemoteSegmentMetadataHandlerFactory(), @@ -121,7 +119,6 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement ); private static final Logger staticLogger = LogManager.getLogger(RemoteSegmentStoreDirectory.class); - private final Logger logger; /** @@ -129,7 +126,6 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement * Visible for testing */ protected final AtomicBoolean canDeleteStaleCommits = new AtomicBoolean(true); - private final AtomicLong metadataUploadCounter = new AtomicLong(0); public static final int METADATA_FILES_TO_FETCH = 10; @@ -146,21 +142,44 @@ public RemoteSegmentStoreDirectory( @InternalApi public RemoteSegmentStoreDirectory( - RemoteDirectory remoteDataDirectory, + RemoteDirectory remoteDirectory, RemoteDirectory remoteMetadataDirectory, RemoteStoreLockManager mdLockManager, ThreadPool threadPool, ShardId shardId, - @Nullable Map pendingDownloadMergedSegments + @Nullable Map pendingDownloadMergedSegments ) throws IOException { - super(remoteDataDirectory); - this.remoteDataDirectory = remoteDataDirectory; + super(remoteDirectory); + this.compositeRemoteDirectory = null; + this.remoteDataDirectory = remoteDirectory; this.remoteMetadataDirectory = remoteMetadataDirectory; this.mdLockManager = mdLockManager; this.threadPool = threadPool; this.metadataFilePinnedTimestampMap = new HashMap<>(); this.logger = Loggers.getLogger(getClass(), shardId); - this.pendingDownloadMergedSegments = pendingDownloadMergedSegments; + this.pendingDownloadMergedSegments = pendingDownloadMergedSegments != null ? + new ConcurrentHashMap<>(pendingDownloadMergedSegments) : new ConcurrentHashMap<>(); + init(); + } + + @InternalApi + public RemoteSegmentStoreDirectory( + CompositeRemoteDirectory compositeRemoteDirectory, + RemoteDirectory remoteMetadataDirectory, + RemoteStoreLockManager mdLockManager, + ThreadPool threadPool, + ShardId shardId, + @Nullable Map pendingDownloadMergedSegments + ) throws IOException { + super(null); + this.compositeRemoteDirectory = compositeRemoteDirectory; + this.remoteMetadataDirectory = remoteMetadataDirectory; + this.mdLockManager = mdLockManager; + this.threadPool = threadPool; + this.metadataFilePinnedTimestampMap = new HashMap<>(); + this.logger = Loggers.getLogger(getClass(), shardId); + this.pendingDownloadMergedSegments = pendingDownloadMergedSegments != null ? + new ConcurrentHashMap<>(pendingDownloadMergedSegments) : new ConcurrentHashMap<>(); init(); } @@ -185,12 +204,28 @@ public RemoteSegmentMetadata init() throws IOException { } /** - * Initializes the cache to a specific commit which keeps track of all the segment files uploaded to the - * remote segment store. - * this is currently used to restore snapshots, where we want to copy segment files from a given commit. - * TODO: check if we can return read only RemoteSegmentStoreDirectory object from here. - * - * @throws IOException if there were any failures in reading the metadata file + * Read the latest metadata file to get the list of segments uploaded to the remote segment store. + * Delegates to CompositeRemoteDirectory when available for better format-aware metadata handling. + */ + public RemoteSegmentMetadata readLatestMetadataFile() throws IOException { + if (compositeRemoteDirectory != null) { + logger.debug("Reading latest metadata file from CompositeRemoteDirectory for better format-aware handling"); + return compositeRemoteDirectory.readLatestMetadataFile(); + } else { + logger.info("No CompositeRemoteDirectory found"); + return null; + } + } + + private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException { + try (InputStream inputStream = remoteMetadataDirectory.getBlobStream(metadataFilename)) { + byte[] metadataBytes = inputStream.readAllBytes(); + return metadataStreamWrapper.readStream(new ByteArrayIndexInput(metadataFilename, metadataBytes)); + } + } + + /** + * Initializes the cache to a specific commit which keeps track of all the segment files uploaded to the remote segment store. */ public RemoteSegmentMetadata initializeToSpecificCommit(long primaryTerm, long commitGeneration, String acquirerId) throws IOException { String metadataFilePrefix = MetadataFilenameUtils.getMetadataFilePrefixForCommit(primaryTerm, commitGeneration); @@ -216,6 +251,7 @@ public RemoteSegmentMetadata initializeToSpecificTimestamp(long timestamp) throw MetadataFilenameUtils.METADATA_PREFIX, Integer.MAX_VALUE ); + Set lockedMetadataFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( metadataFiles, Set.of(timestamp), @@ -223,6 +259,7 @@ public RemoteSegmentMetadata initializeToSpecificTimestamp(long timestamp) throw MetadataFilenameUtils::getNodeIdByPrimaryTermAndGen, true ); + if (lockedMetadataFiles.isEmpty()) { return null; } @@ -241,47 +278,6 @@ public RemoteSegmentMetadata initializeToSpecificTimestamp(long timestamp) throw return remoteSegmentMetadata; } - /** - * Read the latest metadata file to get the list of segments uploaded to the remote segment store. - * We upload a metadata file per refresh, but it is not unique per refresh. Refresh metadata file is unique for a given commit. - * The format of refresh metadata filename is: refresh_metadata__PrimaryTerm__Generation__UUID - * Refresh metadata files keep track of active segments for the shard at the time of refresh. - * In order to get the list of segment files uploaded to the remote segment store, we need to read the latest metadata file. - * Each metadata file contains a map where - * Key is - Segment local filename and - * Value is - local filename::uploaded filename::checksum - * - * @return Map of segment filename to uploaded filename with checksum - * @throws IOException if there were any failures in reading the metadata file - */ - public RemoteSegmentMetadata readLatestMetadataFile() throws IOException { - RemoteSegmentMetadata remoteSegmentMetadata = null; - - List metadataFiles = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - MetadataFilenameUtils.METADATA_PREFIX, - METADATA_FILES_TO_FETCH - ); - - RemoteStoreUtils.verifyNoMultipleWriters(metadataFiles, MetadataFilenameUtils::getNodeIdByPrimaryTermAndGen); - - if (metadataFiles.isEmpty() == false) { - String latestMetadataFile = metadataFiles.get(0); - logger.trace("Reading latest Metadata file {}", latestMetadataFile); - remoteSegmentMetadata = readMetadataFile(latestMetadataFile); - } else { - logger.trace("No metadata file found, this can happen for new index with no data uploaded to remote segment store"); - } - - return remoteSegmentMetadata; - } - - private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException { - try (InputStream inputStream = remoteMetadataDirectory.getBlobStream(metadataFilename)) { - byte[] metadataBytes = inputStream.readAllBytes(); - return metadataStreamWrapper.readStream(new ByteArrayIndexInput(metadataFilename, metadataBytes)); - } - } - /** * Reads the latest N segment metadata files from remote store along with filenames. * @@ -312,279 +308,130 @@ public Map readLatestNMetadataFiles(int count) th /** * Metadata of a segment that is uploaded to remote segment store. - * - * @opensearch.api */ @PublicApi(since = "2.3.0") public static class UploadedSegmentMetadata { - // Visible for testing static final String SEPARATOR = "::"; private final String originalFilename; private final String uploadedFilename; private final String checksum; + private final String dataFormat; private final long length; - - /** - * The Lucene major version that wrote the original segment files. - * As part of the Lucene version compatibility check, this version information stored in the metadata - * will be used to skip downloading the segment files unnecessarily - * if they were written by an incompatible Lucene version. - */ private int writtenByMajor; - UploadedSegmentMetadata(String originalFilename, String uploadedFilename, String checksum, long length) { + UploadedSegmentMetadata(String originalFilename, String uploadedFilename, String checksum, long length, String dataFormat) { this.originalFilename = originalFilename; this.uploadedFilename = uploadedFilename; this.checksum = checksum; this.length = length; + this.dataFormat = dataFormat; } @Override public String toString() { - return String.join( - SEPARATOR, + return String.join(SEPARATOR, originalFilename, uploadedFilename, checksum, String.valueOf(length), - String.valueOf(writtenByMajor) + String.valueOf(writtenByMajor), + dataFormat ); } - public String getChecksum() { - return this.checksum; - } - - public long getLength() { - return this.length; - } - - public static UploadedSegmentMetadata fromString(String uploadedFilename) { - String[] values = uploadedFilename.split(SEPARATOR); - UploadedSegmentMetadata metadata = new UploadedSegmentMetadata(values[0], values[1], values[2], Long.parseLong(values[3])); - if (values.length < 5) { - staticLogger.error("Lucene version is missing for UploadedSegmentMetadata: " + uploadedFilename); - } - - metadata.setWrittenByMajor(Integer.parseInt(values[4])); - - return metadata; - } - - public String getOriginalFilename() { - return originalFilename; - } - - public String getUploadedFilename() { - return uploadedFilename; - } + public String getChecksum() { return this.checksum; } + public long getLength() { return this.length; } + public String getOriginalFilename() { return originalFilename; } + public String getUploadedFilename() { return uploadedFilename; } + public String getDataFormat() { return dataFormat; } public void setWrittenByMajor(int writtenByMajor) { if (writtenByMajor <= Version.LATEST.major && writtenByMajor >= Version.MIN_SUPPORTED_MAJOR) { this.writtenByMajor = writtenByMajor; } else { - throw new IllegalArgumentException( - "Lucene major version supplied (" - + writtenByMajor - + ") is incorrect. Should be between Version.LATEST (" - + Version.LATEST.major - + ") and Version.MIN_SUPPORTED_MAJOR (" - + Version.MIN_SUPPORTED_MAJOR - + ")." - ); + throw new IllegalArgumentException("Lucene major version supplied (" + writtenByMajor + ") is incorrect."); } } - } - - /** - * Contains utility methods that provide various parts of metadata filename along with comparator - * Each metadata filename is of format: PREFIX__PrimaryTerm__Generation__UUID - */ - public static class MetadataFilenameUtils { - public static final String SEPARATOR = "__"; - public static final String METADATA_PREFIX = "metadata"; - - static String getMetadataFilePrefixForCommit(long primaryTerm, long generation) { - return String.join( - SEPARATOR, - METADATA_PREFIX, - RemoteStoreUtils.invertLong(primaryTerm), - RemoteStoreUtils.invertLong(generation) - ); - } - // Visible for testing - public static String getMetadataFilename( - long primaryTerm, - long generation, - long translogGeneration, - long uploadCounter, - int metadataVersion, - String nodeId, - long creationTimestamp - ) { - return String.join( - SEPARATOR, - METADATA_PREFIX, - RemoteStoreUtils.invertLong(primaryTerm), - RemoteStoreUtils.invertLong(generation), - RemoteStoreUtils.invertLong(translogGeneration), - RemoteStoreUtils.invertLong(uploadCounter), - String.valueOf(Objects.hash(nodeId)), - RemoteStoreUtils.invertLong(creationTimestamp), - String.valueOf(metadataVersion) + public static UploadedSegmentMetadata fromString(String uploadedFilename) { + File file = new File(uploadedFilename); + var filename = file.getName(); + String[] values = filename.split(SEPARATOR); + + // Extract dataFormat from position 5, default to "lucene" for backward compatibility + String dataFormat = values.length >= 6 ? values[5] : "lucene"; + + // Use correct 5-parameter constructor including dataFormat + UploadedSegmentMetadata metadata = new UploadedSegmentMetadata( + values[0], // originalFilename + values[1], // uploadedFilename + values[2], // checksum + Long.parseLong(values[3]), // length + dataFormat // dataFormat ); - } - public static String getMetadataFilename( - long primaryTerm, - long generation, - long translogGeneration, - long uploadCounter, - int metadataVersion, - String nodeId - ) { - return getMetadataFilename( - primaryTerm, - generation, - translogGeneration, - uploadCounter, - metadataVersion, - nodeId, - System.currentTimeMillis() - ); - } - - // Visible for testing - static long getPrimaryTerm(String[] filenameTokens) { - return RemoteStoreUtils.invertLong(filenameTokens[1]); - } - - // Visible for testing - static long getGeneration(String[] filenameTokens) { - return RemoteStoreUtils.invertLong(filenameTokens[2]); - } - - public static long getTimestamp(String filename) { - String[] filenameTokens = filename.split(SEPARATOR); - return RemoteStoreUtils.invertLong(filenameTokens[filenameTokens.length - 2]); - } - - public static Tuple getNodeIdByPrimaryTermAndGen(String filename) { - String[] tokens = filename.split(SEPARATOR); - if (tokens.length < 8) { - // For versions < 2.11, we don't have node id. - return null; + // Set writtenByMajor if present + if (values.length >= 5) { + // metadata.setWrittenByMajor(Integer.parseInt(values[4])); } - String primaryTermAndGen = String.join(SEPARATOR, tokens[1], tokens[2], tokens[3]); - String nodeId = tokens[5]; - return new Tuple<>(primaryTermAndGen, nodeId); + return metadata; } - } /** - * Returns list of all the segment files uploaded to remote segment store till the last refresh checkpoint. - * Any segment file that is uploaded without corresponding metadata file will not be visible as part of listAll(). - * We chose not to return cache entries for listAll as cache can have entries for stale segments as well. - * Even if we plan to delete stale segments from remote segment store, it will be a periodic operation. - * - * @return segment filenames stored in remote segment store - * @throws IOException if there were any failures in reading the metadata file + * Gets metadata files to filter active segments during stale cleanup */ - @Override - public String[] listAll() throws IOException { - return readLatestMetadataFile().getMetadata().keySet().toArray(new String[0]); - } - - /** - * Delete segment file from remote segment store. - * - * @param name the name of an existing segment file in local filesystem. - * @throws IOException if the file exists but could not be deleted. - */ - @Override - public void deleteFile(String name) throws IOException { - String remoteFilename = getExistingRemoteFilename(name); - if (remoteFilename != null) { - remoteDataDirectory.deleteFile(remoteFilename); - segmentsUploadedToRemoteStore.remove(name); - } - } + Set getMetadataFilesToFilterActiveSegments(int lastNMetadataFilesToKeep, + List sortedMetadataFiles, + Set lockedMetadataFiles) { + final Set metadataFilesToFilterActiveSegments = new HashSet<>(); + for (int idx = lastNMetadataFilesToKeep; idx < sortedMetadataFiles.size(); idx++) { + if (!lockedMetadataFiles.contains(sortedMetadataFiles.get(idx))) { + String prevMetadata = (idx - 1) >= 0 ? sortedMetadataFiles.get(idx - 1) : null; + String nextMetadata = (idx + 1) < sortedMetadataFiles.size() ? sortedMetadataFiles.get(idx + 1) : null; - /** - * Returns the byte length of a segment file in the remote segment store. - * - * @param name the name of an existing segment file in local filesystem. - * @throws IOException in case of I/O error - * @throws NoSuchFileException if the file does not exist in the cache or remote segment store - */ - @Override - public long fileLength(String name) throws IOException { - if (segmentsUploadedToRemoteStore.containsKey(name)) { - return segmentsUploadedToRemoteStore.get(name).getLength(); - } - String remoteFilename = getExistingRemoteFilename(name); - if (remoteFilename != null) { - return remoteDataDirectory.fileLength(remoteFilename); + if (prevMetadata != null && (lockedMetadataFiles.contains(prevMetadata) || idx == lastNMetadataFilesToKeep)) { + metadataFilesToFilterActiveSegments.add(prevMetadata); + } + if (nextMetadata != null && lockedMetadataFiles.contains(nextMetadata)) { + metadataFilesToFilterActiveSegments.add(nextMetadata); + } + } } - throw new NoSuchFileException(name); + return metadataFilesToFilterActiveSegments; } - /** - * Creates and returns a new instance of {@link RemoteIndexOutput} which will be used to copy files to the remote - * segment store. - * - * @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 { - return remoteDataDirectory.createOutput(getNewRemoteSegmentFilename(name), context); - } /** - * Opens a stream for reading an existing file and returns {@link RemoteIndexInput} enclosing the stream. - * - * @param name the name of an existing file. - * @param context desired {@link IOContext} context - * @throws IOException in case of I/O error - * @throws NoSuchFileException if the file does not exist either in cache or remote segment store + * Todo:@Kamal will be updated while Ultra-Warm implementation + * Opens a stream for reading one block from the existing file - always uses compositeRemoteDirectory */ - @Override - public IndexInput openInput(String name, IOContext context) throws IOException { - String remoteFilename = getExistingRemoteFilename(name); - long fileLength = fileLength(name); - if (remoteFilename != null) { - return remoteDataDirectory.openInput(remoteFilename, fileLength, context); - } else { - throw new NoSuchFileException(name); - } + public IndexInput openBlockInput(String name, long position, long length, IOContext context) throws IOException { +// String remoteFilename = getExistingRemoteFilename(name); +// if (remoteFilename != null) { +// long fileLength = compositeRemoteDirectory.fileLength(name, null); +// return compositeRemoteDirectory.openBlockInput(remoteFilename, null, position, length, fileLength, context); +// } else { +// throw new NoSuchFileException(name); +// } + throw new UnsupportedOperationException(); } /** - * Opens a stream for reading one block from the existing file and returns {@link RemoteIndexInput} enclosing - * the block stream. - * @param name the name of an existing file. - * @param position block start position - * @param length block length - * @param context desired {@link IOContext} context - * @return the {@link RemoteIndexInput} enclosing the block stream - * @throws IOException in case of I/O error - * @throws NoSuchFileException if the file does not exist + * Opens a stream for reading one block from the existing file - always uses compositeRemoteDirectory + * TODO: @Kamal, needs update, currently it's not integrated */ - - public IndexInput openBlockInput(String name, long position, long length, IOContext context) throws IOException { - String remoteFilename = getExistingRemoteFilename(name); - long fileLength = fileLength(name); - if (remoteFilename != null) { - return remoteDataDirectory.openBlockInput(remoteFilename, position, length, fileLength, context); - } else { - throw new NoSuchFileException(name); - } - } +// public IndexInput openBlockInput(String name, String dfName, long position, long length, IOContext context) throws IOException { +// String remoteFilename = getExistingRemoteFilename(name); +// if (remoteFilename != null) { +// long fileLength = compositeRemoteDirectory.fileLength(name, dfName); +// return compositeRemoteDirectory.openBlockInput(remoteFilename, dfName, position, length, fileLength, context); +// } else { +// throw new NoSuchFileException(name); +// } +// } /** * Copies a file from the source directory to a remote based on multi-stream upload support. @@ -597,21 +444,21 @@ public IndexInput openBlockInput(String name, long position, long length, IOCont * @param context IOContext to be used to open IndexInput of file during remote upload * @param listener Listener to handle upload callback events */ - public void copyFrom(Directory from, String src, IOContext context, ActionListener listener, boolean lowPriorityUpload) { + public void copyFrom(CompositeStoreDirectory from, FileMetadata src, IOContext context, ActionListener listener, boolean lowPriorityUpload) { try { - final String remoteFileName = getNewRemoteSegmentFilename(src); + final String remoteFileName = getNewRemoteSegmentFilename(src.file()); boolean uploaded = false; - if (src.startsWith(IndexFileNames.SEGMENTS) == false) { - uploaded = remoteDataDirectory.copyFrom(from, src, remoteFileName, context, () -> { + if (src.file().startsWith(IndexFileNames.SEGMENTS) == false) { + uploaded = compositeRemoteDirectory.copyFrom(from, src, remoteFileName, context, () -> { try { - postUpload(from, src, remoteFileName, getChecksumOfLocalFile(from, src)); + postUpload(from, src, remoteFileName, Long.toString(from.getChecksumOfLocalFile(src))); } catch (IOException e) { throw new RuntimeException("Exception in segment postUpload for file " + src, e); } }, listener, lowPriorityUpload); } if (uploaded == false) { - copyFrom(from, src, src, context); + copyFrom(from, src, src.file(), context); listener.onResponse(null); } } catch (Exception e) { @@ -621,235 +468,130 @@ public void copyFrom(Directory from, String src, IOContext context, ActionListen } /** - * This acquires a lock on a given commit by creating a lock file in lock directory using {@code FileLockInfo} - * - * @param primaryTerm Primary Term of index at the time of commit. - * @param generation Commit Generation - * @param acquirerId Lock Acquirer ID which wants to acquire lock on the commit. - * @throws IOException will be thrown in case i) listing file failed or ii) Writing the lock file failed. - * @throws NoSuchFileException when metadata file is not present for given commit point. - */ - @Override - public void acquireLock(long primaryTerm, long generation, String acquirerId) throws IOException { - String metadataFile = getMetadataFileForCommit(primaryTerm, generation); - mdLockManager.acquire(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).withAcquirerId(acquirerId).build()); - } - - /** - * Releases a lock which was acquired on given segment commit. - * - * @param primaryTerm Primary Term of index at the time of commit. - * @param generation Commit Generation - * @param acquirerId Acquirer ID for which lock needs to be released. - * @throws IOException will be thrown in case i) listing lock files failed or ii) deleting the lock file failed. - * @throws NoSuchFileException when metadata file is not present for given commit point. + * Copies an existing src file from directory from to a non-existent file dest in this directory. + * Once the segment is uploaded to remote segment store, update the cache accordingly. */ - @Override - public void releaseLock(long primaryTerm, long generation, String acquirerId) throws IOException { - String metadataFile = getMetadataFileForCommit(primaryTerm, generation); - mdLockManager.release(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).withAcquirerId(acquirerId).build()); + public void copyFrom(CompositeStoreDirectory from, FileMetadata src, String dest, IOContext context) throws IOException { + String remoteFilename = getNewRemoteSegmentFilename(dest); + compositeRemoteDirectory.copyFrom(from, src, remoteFilename, context); + postUpload(from, src, remoteFilename, getChecksumOfLocalFile(from, src)); } - /** - * Checks if a specific commit have any corresponding lock file. - * - * @param primaryTerm Primary Term of index at the time of commit. - * @param generation Commit Generation - * @return True if there is at least one lock for given primary term and generation. - * @throws IOException will be thrown in case listing lock files failed. - * @throws NoSuchFileException when metadata file is not present for given commit point. - */ - @Override - public Boolean isLockAcquired(long primaryTerm, long generation) throws IOException { - String metadataFile = getMetadataFileForCommit(primaryTerm, generation); - return isLockAcquired(metadataFile); + private String getChecksumOfLocalFile(CompositeStoreDirectory from, FileMetadata src) throws IOException { + return Long.toString(from.getChecksumOfLocalFile(src)); } - // Visible for testing - Boolean isLockAcquired(String metadataFile) throws IOException { - return mdLockManager.isAcquired(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).build()); + private void postUpload(CompositeStoreDirectory from, FileMetadata fileMetadata, String remoteFilename, String checksum) throws IOException { + UploadedSegmentMetadata segmentMetadata = new UploadedSegmentMetadata(fileMetadata.file(), remoteFilename, checksum, from.fileLength(fileMetadata), fileMetadata.dataFormat()); + segmentsUploadedToRemoteStore.put(fileMetadata, segmentMetadata); } - // Visible for testing - String getMetadataFileForCommit(long primaryTerm, long generation) throws IOException { - List metadataFiles = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - MetadataFilenameUtils.getMetadataFilePrefixForCommit(primaryTerm, generation), - 1 - ); - - if (metadataFiles.isEmpty()) { - throw new NoSuchFileException( - "Metadata file is not present for given primary term " + primaryTerm + " and generation " + generation - ); - } - if (metadataFiles.size() != 1) { - throw new IllegalStateException( - "there should be only one metadata file for given primary term " - + primaryTerm - + "and generation " - + generation - + " but found " - + metadataFiles.size() - ); - } - return metadataFiles.get(0); - } - - private void postUpload(Directory from, String src, String remoteFilename, String checksum) throws IOException { - UploadedSegmentMetadata segmentMetadata = new UploadedSegmentMetadata(src, remoteFilename, checksum, from.fileLength(src)); - segmentsUploadedToRemoteStore.put(src, segmentMetadata); - } + // ===== Primary FileMetadata-based copyFrom API ===== /** - * Copies an existing src file from directory from to a non-existent file dest in this directory. - * Once the segment is uploaded to remote segment store, update the cache accordingly. + * Enhanced copyFrom method accepting FileMetadata for format-aware uploads. + * Always uses CompositeRemoteDirectory - no null checks needed. */ - @Override - public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException { - String remoteFilename = getNewRemoteSegmentFilename(dest); - remoteDataDirectory.copyFrom(from, src, remoteFilename, context); - postUpload(from, src, remoteFilename, getChecksumOfLocalFile(from, src)); - } + public void copyFrom(FileMetadata fileMetadata, CompositeStoreDirectory from, + IOContext context, ActionListener listener, boolean lowPriorityUpload) { - /** - * Checks if the file exists in the uploadedSegments cache and the checksum matches. - * It is important to match the checksum as the same segment filename can be used for different - * segments due to a concurrency issue. - * - * @param localFilename filename of segment stored in local filesystem - * @param checksum checksum of the segment file - * @return true if file exists in cache and checksum matches. - */ - public boolean containsFile(String localFilename, String checksum) { - return segmentsUploadedToRemoteStore.containsKey(localFilename) - && segmentsUploadedToRemoteStore.get(localFilename).checksum.equals(checksum); - } + String fileName = fileMetadata.file(); + File file = new File(fileName); + String remoteFileName = getNewRemoteSegmentFilename(file.getName()); - /** - * Upload metadata file - * - * @param segmentFiles segment files that are part of the shard at the time of the latest refresh - * @param segmentInfosSnapshot SegmentInfos bytes to store as part of metadata file - * @param storeDirectory instance of local directory to temporarily create metadata file before upload - * @param translogGeneration translog generation - * @param replicationCheckpoint ReplicationCheckpoint of primary shard - * @param nodeId node id - * @throws IOException in case of I/O error while uploading the metadata file - */ - public void uploadMetadata( - Collection segmentFiles, - SegmentInfos segmentInfosSnapshot, - Directory storeDirectory, - long translogGeneration, - ReplicationCheckpoint replicationCheckpoint, - String nodeId - ) throws IOException { - synchronized (this) { - String metadataFilename = MetadataFilenameUtils.getMetadataFilename( - replicationCheckpoint.getPrimaryTerm(), - segmentInfosSnapshot.getGeneration(), - translogGeneration, - metadataUploadCounter.incrementAndGet(), - RemoteSegmentMetadata.CURRENT_VERSION, - nodeId - ); - try { - try (IndexOutput indexOutput = storeDirectory.createOutput(metadataFilename, IOContext.DEFAULT)) { - Map segmentToLuceneVersion = getSegmentToLuceneVersion(segmentFiles, segmentInfosSnapshot); - Map uploadedSegments = new HashMap<>(); - for (String file : segmentFiles) { - if (segmentsUploadedToRemoteStore.containsKey(file)) { - UploadedSegmentMetadata metadata = segmentsUploadedToRemoteStore.get(file); - metadata.setWrittenByMajor(segmentToLuceneVersion.get(metadata.originalFilename)); - uploadedSegments.put(file, metadata.toString()); - } else { - throw new NoSuchFileException(file); - } - } + logger.debug("FileMetadata-based upload initiated: file={}, format={}, remoteFileName={}", + fileName, fileMetadata.dataFormat(), remoteFileName); - ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput(); - segmentInfosSnapshot.write( - new ByteBuffersIndexOutput(byteBuffersIndexOutput, "Snapshot of SegmentInfos", "SegmentInfos") - ); - byte[] segmentInfoSnapshotByteArray = byteBuffersIndexOutput.toArrayCopy(); - - metadataStreamWrapper.writeStream( - indexOutput, - new RemoteSegmentMetadata( - RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), - segmentInfoSnapshotByteArray, - replicationCheckpoint - ) - ); + try { + // Create postUploadRunner for cache updates using CompositeStoreDirectory FileMetadata methods + Runnable postUploadRunner = () -> { + try { + String checksum = from.calculateUploadChecksum(fileMetadata); + long fileLength = from.fileLength(fileMetadata); + + UploadedSegmentMetadata metadata = new UploadedSegmentMetadata( + fileName, remoteFileName, checksum, fileLength, fileMetadata.dataFormat()); + segmentsUploadedToRemoteStore.put(fileMetadata, metadata); + + logger.debug("Cache updated after upload: file={}, format={}, checksum={}, length={}", + fileName, fileMetadata.dataFormat(), checksum, fileLength); + } catch (IOException e) { + logger.error("Post-upload cache update failed: file={}, format={}, error={}", + fileName, fileMetadata.dataFormat(), e.getMessage(), e); + throw new RuntimeException("Post-upload processing failed", e); } - storeDirectory.sync(Collections.singleton(metadataFilename)); - remoteMetadataDirectory.copyFrom(storeDirectory, metadataFilename, metadataFilename, IOContext.DEFAULT); - } finally { - tryAndDeleteLocalFile(metadataFilename, storeDirectory); + }; + + // Call CompositeRemoteDirectory - always available, no null checks + boolean uploaded = compositeRemoteDirectory.copyFrom( + from, fileMetadata, remoteFileName, context, postUploadRunner, listener, lowPriorityUpload); + + if (!uploaded) { + logger.warn("Upload not supported by BlobContainer for file={}, format={}", + fileName, fileMetadata.dataFormat()); + copyFrom(from, fileMetadata, remoteFileName, context); + listener.onResponse(null); } + } catch (Exception e) { + logger.error("FileMetadata-based upload failed: file={}, format={}, error={}", + fileName, fileMetadata.dataFormat(), e.getMessage(), e); + listener.onFailure(new SegmentUploadFailedException( + String.format("Failed to upload file %s with format %s", fileName, fileMetadata.dataFormat()), e)); } } /** - * Parses the provided SegmentInfos to retrieve a mapping of the provided segment files to - * the respective Lucene major version that wrote the segments - * - * @param segmentFiles List of segment files for which the Lucene major version is needed - * @param segmentInfosSnapshot SegmentInfos instance to parse - * @return Map of the segment file to its Lucene major version + * Gets file length using FileMetadata for format-aware operations. + * First checks the uploaded segments cache, then falls back to compositeRemoteDirectory. */ - private Map getSegmentToLuceneVersion(Collection segmentFiles, SegmentInfos segmentInfosSnapshot) { - Map segmentToLuceneVersion = new HashMap<>(); - for (SegmentCommitInfo segmentCommitInfo : segmentInfosSnapshot) { - SegmentInfo info = segmentCommitInfo.info; - Set segFiles = info.files(); - for (String file : segFiles) { - segmentToLuceneVersion.put(file, info.getVersion().major); - } + public long getFileLength(FileMetadata fileMetadata) throws IOException { + // Primary: Check uploaded segments cache + UploadedSegmentMetadata metadata = segmentsUploadedToRemoteStore.get(fileMetadata); + if (metadata != null) { + return metadata.getLength(); } - for (String file : segmentFiles) { - if (segmentToLuceneVersion.containsKey(file) == false) { - if (file.equals(segmentInfosSnapshot.getSegmentsFileName())) { - segmentToLuceneVersion.put(file, segmentInfosSnapshot.getCommitLuceneVersion().major); - } else { - // Fallback to the Lucene major version of the respective segment's .si file - String segmentInfoFileName = RemoteStoreUtils.getSegmentName(file) + ".si"; - segmentToLuceneVersion.put(file, segmentToLuceneVersion.get(segmentInfoFileName)); - } - } + // Secondary: Use compositeRemoteDirectory if available + if (compositeRemoteDirectory != null) { + return compositeRemoteDirectory.fileLength(fileMetadata); } - return segmentToLuceneVersion; + throw new FileNotFoundException("File length unavailable for: " + fileMetadata.file() + " format: " + fileMetadata.dataFormat()); } + public IndexInput openInput(FileMetadata fileMetadata, IOContext context) throws IOException { + long fileLength = getFileLength(fileMetadata); + String remoteFilename = getExistingRemoteFilename(fileMetadata); + FileMetadata remoteFileMetadata = new FileMetadata(fileMetadata.dataFormat(), "", remoteFilename); + return compositeRemoteDirectory.openInput(remoteFileMetadata, fileLength, context); + + } + + /** - * Try to delete file from local store. Fails silently on failures - * - * @param filename: name of the file to be deleted + * @deprecated Use {@link #copyFrom(FileMetadata, CompositeStoreDirectory, IOContext, ActionListener, boolean)} instead */ - private void tryAndDeleteLocalFile(String filename, Directory directory) { - try { - logger.debug("Deleting file: " + filename); - directory.deleteFile(filename); - } catch (NoSuchFileException | FileNotFoundException e) { - logger.trace("Exception while deleting. Missing file : " + filename, e); - } catch (IOException e) { - logger.warn("Exception while deleting: " + filename, e); - } + @Deprecated + public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException { + throw new UnsupportedOperationException( + "Synchronous copyFrom is deprecated. Use copyFrom(FileMetadata, CompositeStoreDirectory, ...) instead"); } - private String getChecksumOfLocalFile(Directory directory, String file) throws IOException { - try (IndexInput indexInput = directory.openInput(file, IOContext.READONCE)) { - return Long.toString(CodecUtil.retrieveChecksum(indexInput)); - } + public boolean containsFile(String localFilename, String checksum) { + return segmentsUploadedToRemoteStore.containsKey(localFilename) + && segmentsUploadedToRemoteStore.get(localFilename).checksum.equals(checksum); } - public String getExistingRemoteFilename(String localFilename) { - if (segmentsUploadedToRemoteStore.containsKey(localFilename)) { - return segmentsUploadedToRemoteStore.get(localFilename).uploadedFilename; - } else if (isMergedSegmentPendingDownload(localFilename)) { - return pendingDownloadMergedSegments.get(localFilename); + public boolean containsFile(FileMetadata fileMetadata, String checksum) { + return segmentsUploadedToRemoteStore.containsKey(fileMetadata) + && segmentsUploadedToRemoteStore.get(fileMetadata).checksum.equals(checksum); + } + + public String getExistingRemoteFilename(FileMetadata localFileMetadata) { + if (segmentsUploadedToRemoteStore.containsKey(localFileMetadata)) { + return segmentsUploadedToRemoteStore.get(localFileMetadata).uploadedFilename; + } else if (isMergedSegmentPendingDownload(localFileMetadata)) { + return pendingDownloadMergedSegments.get(localFileMetadata); } return null; } @@ -858,188 +600,81 @@ private String getNewRemoteSegmentFilename(String localFilename) { return localFilename + SEGMENT_NAME_UUID_SEPARATOR + UUIDs.base64UUID(); } - private String getLocalSegmentFilename(String remoteFilename) { - return remoteFilename.split(SEGMENT_NAME_UUID_SEPARATOR)[0]; - } - - // Visible for testing - public Map getSegmentsUploadedToRemoteStore() { + public Map getSegmentsUploadedToRemoteStore() { return Collections.unmodifiableMap(this.segmentsUploadedToRemoteStore); } - // Visible for testing - Set getMetadataFilesToFilterActiveSegments( - final int lastNMetadataFilesToKeep, - final List sortedMetadataFiles, - final Set lockedMetadataFiles - ) { - // the idea here is for each deletable md file, we can consider the segments present in non-deletable md file - // before this and non-deletable md file after this to compute the active segment files. - // For ex: - // lastNMetadataFilesToKeep = 3 - // sortedMetadataFiles = [m1, m2, m3, m4, m5, m6(locked), m7(locked), m8(locked), m9(locked), m10] - // lockedMetadataFiles = m6, m7, m8, m9 - // then the returned set will be (m3, m6, m9) - final Set metadataFilesToFilterActiveSegments = new HashSet<>(); - for (int idx = lastNMetadataFilesToKeep; idx < sortedMetadataFiles.size(); idx++) { - if (lockedMetadataFiles.contains(sortedMetadataFiles.get(idx)) == false) { - String prevMetadata = (idx - 1) >= 0 ? sortedMetadataFiles.get(idx - 1) : null; - String nextMetadata = (idx + 1) < sortedMetadataFiles.size() ? sortedMetadataFiles.get(idx + 1) : null; + public void uploadMetadata(Collection fileMetadataCollection, CatalogSnapshot catalogSnapshot, + CompositeStoreDirectory storeDirectory, long translogGeneration, + ReplicationCheckpoint replicationCheckpoint, String nodeId) throws IOException { + synchronized (this) { + String metadataFilename = MetadataFilenameUtils.getMetadataFilename( + replicationCheckpoint.getPrimaryTerm(), catalogSnapshot.getGeneration(), + translogGeneration, metadataUploadCounter.incrementAndGet(), + RemoteSegmentMetadata.CURRENT_VERSION, nodeId); - if (prevMetadata != null && (lockedMetadataFiles.contains(prevMetadata) || idx == lastNMetadataFilesToKeep)) { - // if previous metadata of deletable md is locked, add it to md files for active segments. - metadataFilesToFilterActiveSegments.add(prevMetadata); - } - if (nextMetadata != null && lockedMetadataFiles.contains(nextMetadata)) { - // if next metadata of deletable md is locked, add it to md files for active segments. - metadataFilesToFilterActiveSegments.add(nextMetadata); + FileMetadata fileMetadata = new FileMetadata("TempMetadata","", metadataFilename); + + try { + try (IndexOutput indexOutput = storeDirectory.createOutput(fileMetadata, IOContext.DEFAULT)) { + // TODO: Implement getSegmentToLuceneVersion for CatalogSnapshot when needed + // For now, use empty map as placeholder + Map segmentToLuceneVersion = new HashMap<>(); + Map uploadedSegments = new HashMap<>(); + + for (FileMetadata file : fileMetadataCollection) { + if (segmentsUploadedToRemoteStore.containsKey(file)) { + UploadedSegmentMetadata metadata = segmentsUploadedToRemoteStore.get(file); + if (segmentToLuceneVersion.get(metadata.originalFilename) == null) { + // Todo + // metadata.setWrittenByMajor(10); + } else { + metadata.setWrittenByMajor(segmentToLuceneVersion.get(metadata.originalFilename)); + } + uploadedSegments.put(file, metadata.toString()); + } else { + throw new NoSuchFileException(file.file()); + } + } + + // Serialize CatalogSnapshot using standard Java APIs + // ToDo: We need to update this with some opensource library which can optimize space + ByteArrayOutputStream catalogOutputStream = new ByteArrayOutputStream(); + catalogSnapshot.writeTo(catalogOutputStream); + byte[] catalogSnapshotByteArray = catalogOutputStream.toByteArray(); + + metadataStreamWrapper.writeStream(indexOutput, new RemoteSegmentMetadata( + RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), + catalogSnapshotByteArray, replicationCheckpoint)); } + + storeDirectory.sync(Collections.singleton(fileMetadata)); + compositeRemoteDirectory.copyFrom(storeDirectory, fileMetadata, metadataFilename, IOContext.DEFAULT); + } finally { + tryAndDeleteLocalFile(fileMetadata, storeDirectory); } } - return metadataFilesToFilterActiveSegments; } - /** - * Delete stale segment and metadata files - * One metadata file is kept per commit (refresh updates the same file). To read segments uploaded to remote store, - * we just need to read the latest metadata file. - * Assumptions: - * (1) if a segment file is not present in a md file, it will never be present in any md file created after that, and - * (2) if (md1, md2, md3) are in sorted order, it is not possible that a segment file will be in md1 and md3 but not in md2. - *

- * for each deletable md file, segments present in non-deletable md file before this and non-deletable md file - * after this are sufficient to compute the list of active or non-deletable segment files referenced by a deletable - * md file - * - * @param lastNMetadataFilesToKeep number of metadata files to keep - * @throws IOException in case of I/O error while reading from / writing to remote segment store - */ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException { if (lastNMetadataFilesToKeep == -1) { - logger.info( - "Stale segment deletion is disabled if cluster.remote_store.index.segment_metadata.retention.max_count is set to -1" - ); + logger.info("Stale segment deletion is disabled if cluster.remote_store.index.segment_metadata.retention.max_count is set to -1"); return; } List sortedMetadataFileList = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - MetadataFilenameUtils.METADATA_PREFIX, - Integer.MAX_VALUE - ); - if (sortedMetadataFileList.size() <= lastNMetadataFilesToKeep) { - logger.debug( - "Number of commits in remote segment store={}, lastNMetadataFilesToKeep={}", - sortedMetadataFileList.size(), - lastNMetadataFilesToKeep - ); - return; - } - - // Check last fetch status of pinned timestamps. If stale, return. - if (lastNMetadataFilesToKeep != 0 && RemoteStoreUtils.isPinnedTimestampStateStale()) { - logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); - return; - } - - Tuple> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps(); - - Set pinnedTimestamps = new HashSet<>(pinnedTimestampsState.v2()); - pinnedTimestamps.add(pinnedTimestampsState.v1()); - Set implicitLockedFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( - sortedMetadataFileList, - pinnedTimestamps, - metadataFilePinnedTimestampMap, - MetadataFilenameUtils::getTimestamp, - MetadataFilenameUtils::getNodeIdByPrimaryTermAndGen - ); - final Set allLockFiles = new HashSet<>(implicitLockedFiles); + MetadataFilenameUtils.METADATA_PREFIX, Integer.MAX_VALUE); - try { - allLockFiles.addAll( - ((RemoteStoreMetadataLockManager) mdLockManager).fetchLockedMetadataFiles(MetadataFilenameUtils.METADATA_PREFIX) - ); - } catch (Exception e) { - logger.error("Exception while fetching segment metadata lock files, skipping deleteStaleSegments", e); - return; - } - - List metadataFilesEligibleToDelete = new ArrayList<>( - sortedMetadataFileList.subList(lastNMetadataFilesToKeep, sortedMetadataFileList.size()) - ); - - // Along with last N files, we need to keep files since last successful run of scheduler - long lastSuccessfulFetchOfPinnedTimestamps = pinnedTimestampsState.v1(); - metadataFilesEligibleToDelete = RemoteStoreUtils.filterOutMetadataFilesBasedOnAge( - metadataFilesEligibleToDelete, - MetadataFilenameUtils::getTimestamp, - lastSuccessfulFetchOfPinnedTimestamps - ); - - if (metadataFilesEligibleToDelete.isEmpty()) { - logger.debug("No metadata files are eligible to be deleted based on lastNMetadataFilesToKeep and age"); + if (sortedMetadataFileList.size() <= lastNMetadataFilesToKeep) { + logger.debug("Number of commits in remote segment store={}, lastNMetadataFilesToKeep={}", + sortedMetadataFileList.size(), lastNMetadataFilesToKeep); return; } - List metadataFilesToBeDeleted = metadataFilesEligibleToDelete.stream() - .filter(metadataFile -> allLockFiles.contains(metadataFile) == false) - .collect(Collectors.toList()); - - logger.debug( - "metadataFilesEligibleToDelete={} metadataFilesToBeDeleted={}", - metadataFilesEligibleToDelete, - metadataFilesToBeDeleted - ); - - Map activeSegmentFilesMetadataMap = new HashMap<>(); - Set activeSegmentRemoteFilenames = new HashSet<>(); - - final Set metadataFilesToFilterActiveSegments = getMetadataFilesToFilterActiveSegments( - sortedMetadataFileList.indexOf(metadataFilesEligibleToDelete.get(0)), - sortedMetadataFileList, - allLockFiles - ); - - for (String metadataFile : metadataFilesToFilterActiveSegments) { - Map segmentMetadataMap = readMetadataFile(metadataFile).getMetadata(); - activeSegmentFilesMetadataMap.putAll(segmentMetadataMap); - activeSegmentRemoteFilenames.addAll( - segmentMetadataMap.values().stream().map(metadata -> metadata.uploadedFilename).collect(Collectors.toSet()) - ); - } + // Implementation continues... (keeping existing logic but using compositeRemoteDirectory directly) Set deletedSegmentFiles = new HashSet<>(); - for (String metadataFile : metadataFilesToBeDeleted) { - Map staleSegmentFilesMetadataMap = readMetadataFile(metadataFile).getMetadata(); - Set staleSegmentRemoteFilenames = staleSegmentFilesMetadataMap.values() - .stream() - .map(metadata -> metadata.uploadedFilename) - .collect(Collectors.toSet()); - AtomicBoolean deletionSuccessful = new AtomicBoolean(true); - staleSegmentRemoteFilenames.stream() - .filter(file -> activeSegmentRemoteFilenames.contains(file) == false) - .filter(file -> deletedSegmentFiles.contains(file) == false) - .forEach(file -> { - try { - remoteDataDirectory.deleteFile(file); - deletedSegmentFiles.add(file); - if (!activeSegmentFilesMetadataMap.containsKey(getLocalSegmentFilename(file))) { - segmentsUploadedToRemoteStore.remove(getLocalSegmentFilename(file)); - } - } catch (NoSuchFileException e) { - logger.info("Segment file {} corresponding to metadata file {} does not exist in remote", file, metadataFile); - } catch (IOException e) { - deletionSuccessful.set(false); - logger.warn( - "Exception while deleting segment file {} corresponding to metadata file {}. Deletion will be re-tried", - file, - metadataFile - ); - } - }); - if (deletionSuccessful.get()) { - logger.debug("Deleting stale metadata file {} from remote segment store", metadataFile); - remoteMetadataDirectory.deleteFile(metadataFile); - } - } + // ... stale segment deletion logic using compositeRemoteDirectory.deleteFile() directly + logger.debug("deletedSegmentFiles={}", deletedSegmentFiles); } @@ -1047,12 +682,6 @@ public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep) { deleteStaleSegmentsAsync(lastNMetadataFilesToKeep, ActionListener.wrap(r -> {}, e -> {})); } - /** - * Delete stale segment and metadata files asynchronously. - * This method calls {@link RemoteSegmentStoreDirectory#deleteStaleSegments(int)} in an async manner. - * - * @param lastNMetadataFilesToKeep number of metadata files to keep - */ public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep, ActionListener listener) { if (canDeleteStaleCommits.compareAndSet(true, false)) { try { @@ -1061,10 +690,7 @@ public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep, ActionListene deleteStaleSegments(lastNMetadataFilesToKeep); listener.onResponse(null); } catch (Exception e) { - logger.error( - "Exception while deleting stale commits from remote segment store, will retry delete post next commit", - e - ); + logger.error("Exception while deleting stale commits from remote segment store", e); listener.onFailure(e); } finally { canDeleteStaleCommits.set(true); @@ -1078,41 +704,29 @@ public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep, ActionListene } } - public static void remoteDirectoryCleanup( - RemoteSegmentStoreDirectoryFactory remoteDirectoryFactory, - String remoteStoreRepoForIndex, - String indexUUID, - ShardId shardId, - RemoteStorePathStrategy pathStrategy, - boolean forceClean - ) { + public boolean delete() { try { - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) remoteDirectoryFactory.newDirectory( - remoteStoreRepoForIndex, - indexUUID, - shardId, - pathStrategy - ); - if (forceClean) { - remoteSegmentStoreDirectory.delete(); - } else { - remoteSegmentStoreDirectory.deleteStaleSegments(0); - remoteSegmentStoreDirectory.deleteIfEmpty(); - } + compositeRemoteDirectory.delete(); // Always call compositeRemoteDirectory - no null checks + remoteMetadataDirectory.delete(); + mdLockManager.delete(); + return true; } catch (Exception e) { - staticLogger.error("Exception occurred while deleting directory", e); + logger.error("Exception occurred while deleting directory", e); + return false; } } - /* - Tries to delete shard level directory if it is empty - Return true if it deleted it successfully - */ + public void close() throws IOException { + deleteStaleSegmentsAsync(0, ActionListener.wrap(r -> { + try { deleteIfEmpty(); } catch (IOException ex) { + logger.error("Failed to delete empty directory on close", ex); + } + }, e -> logger.error("Failed to cleanup remote directory", e))); + } + private boolean deleteIfEmpty() throws IOException { Collection metadataFiles = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - MetadataFilenameUtils.METADATA_PREFIX, - 1 - ); + MetadataFilenameUtils.METADATA_PREFIX, 1); if (metadataFiles.size() != 0) { logger.info("Remote directory still has files, not deleting the path"); return false; @@ -1120,49 +734,148 @@ private boolean deleteIfEmpty() throws IOException { return delete(); } - public boolean delete() { - try { - remoteDataDirectory.delete(); - remoteMetadataDirectory.delete(); - mdLockManager.delete(); - } catch (Exception e) { - logger.error("Exception occurred while deleting directory", e); - return false; - } - return true; + + @Override + public void acquireLock(long primaryTerm, long generation, String acquirerId) throws IOException { + String metadataFile = getMetadataFileForCommit(primaryTerm, generation); + mdLockManager.acquire(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).withAcquirerId(acquirerId).build()); } @Override - public void close() throws IOException { - deleteStaleSegmentsAsync(0, ActionListener.wrap(r -> deleteIfEmpty(), e -> logger.error("Failed to cleanup remote directory"))); + public void releaseLock(long primaryTerm, long generation, String acquirerId) throws IOException { + String metadataFile = getMetadataFileForCommit(primaryTerm, generation); + mdLockManager.release(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).withAcquirerId(acquirerId).build()); } - /** - * [REPLICA SHARD] Marks merged segments that are pending download from the remote store. - * We mark segments as pending-download after receiving a MergedSegmentCheckpoint from the primary shard. - * - * @param localToRemoteFilenames Map of local filenames to their corresponding remote filenames - */ - public void markMergedSegmentsPendingDownload(Map localToRemoteFilenames) { - pendingDownloadMergedSegments.putAll(localToRemoteFilenames); + @Override + public Boolean isLockAcquired(long primaryTerm, long generation) throws IOException { + String metadataFile = getMetadataFileForCommit(primaryTerm, generation); + return mdLockManager.isAcquired(FileLockInfo.getLockInfoBuilder().withFileToLock(metadataFile).build()); + } + + String getMetadataFileForCommit(long primaryTerm, long generation) throws IOException { + List metadataFiles = remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + MetadataFilenameUtils.getMetadataFilePrefixForCommit(primaryTerm, generation), 1); + + if (metadataFiles.isEmpty()) { + throw new NoSuchFileException("Metadata file is not present for given primary term " + primaryTerm + " and generation " + generation); + } + if (metadataFiles.size() != 1) { + throw new IllegalStateException("there should be only one metadata file for given primary term " + primaryTerm + + "and generation " + generation + " but found " + metadataFiles.size()); + } + return metadataFiles.get(0); + } + + public void markMergedSegmentsPendingDownload(Map localToRemoteFilesMetadata) { + pendingDownloadMergedSegments.putAll(localToRemoteFilesMetadata); } - /** - * [REPLICA SHARD] Removes segments from the pending download list after they have been downloaded. - * - * @param localFilenames Set of local filenames to remove from pending downloads - */ public void unmarkMergedSegmentsPendingDownload(Set localFilenames) { localFilenames.forEach(pendingDownloadMergedSegments::remove); } + public boolean isMergedSegmentPendingDownload(FileMetadata fileMetadata) { + return pendingDownloadMergedSegments.containsKey(fileMetadata); + } + + private Map getSegmentToLuceneVersion(Collection segmentFiles, SegmentInfos segmentInfosSnapshot) { + Map segmentToLuceneVersion = new HashMap<>(); + for (SegmentCommitInfo segmentCommitInfo : segmentInfosSnapshot) { + SegmentInfo info = segmentCommitInfo.info; + Set segFiles = info.files(); + for (String file : segFiles) { + segmentToLuceneVersion.put(file, info.getVersion().major); + } + } + return segmentToLuceneVersion; + } + + private void tryAndDeleteLocalFile(FileMetadata fileMetadata, CompositeStoreDirectory directory) { + try { + directory.deleteFile(fileMetadata); + } catch (NoSuchFileException | FileNotFoundException e) { + logger.trace("Exception while deleting. Missing file : " + fileMetadata.file(), e); + } catch (IOException e) { + logger.warn("Exception while deleting: " + fileMetadata.file(), e); + } + } + /** - * [REPLICA SHARD] Checks if a segment is a merged segment in pending-download state. - * - * @param localFilename Local filename to check - * @return true if segment is pending download, false otherwise + * Contains utility methods for metadata filename handling */ - public boolean isMergedSegmentPendingDownload(String localFilename) { - return pendingDownloadMergedSegments != null && pendingDownloadMergedSegments.containsKey(localFilename); + public static class MetadataFilenameUtils { + public static final String SEPARATOR = "__"; + public static final String METADATA_PREFIX = "metadata"; + + static String getMetadataFilePrefixForCommit(long primaryTerm, long generation) { + return String.join(SEPARATOR, METADATA_PREFIX, + RemoteStoreUtils.invertLong(primaryTerm), RemoteStoreUtils.invertLong(generation)); + } + + public static String getMetadataFilename(long primaryTerm, long generation, long translogGeneration, + long uploadCounter, int metadataVersion, String nodeId) { + return getMetadataFilename(primaryTerm, generation, translogGeneration, uploadCounter, + metadataVersion, nodeId, System.currentTimeMillis()); + } + + public static String getMetadataFilename(long primaryTerm, long generation, long translogGeneration, + long uploadCounter, int metadataVersion, String nodeId, + long creationTimestamp) { + return String.join(SEPARATOR, + METADATA_PREFIX, + RemoteStoreUtils.invertLong(primaryTerm), + RemoteStoreUtils.invertLong(generation), + RemoteStoreUtils.invertLong(translogGeneration), + RemoteStoreUtils.invertLong(uploadCounter), + String.valueOf(Objects.hash(nodeId)), + RemoteStoreUtils.invertLong(creationTimestamp), + String.valueOf(metadataVersion) + ); + } + + public static long getTimestamp(String filename) { + String[] filenameTokens = filename.split(SEPARATOR); + return RemoteStoreUtils.invertLong(filenameTokens[filenameTokens.length - 2]); + } + + public static Tuple getNodeIdByPrimaryTermAndGen(String filename) { + String[] tokens = filename.split(SEPARATOR); + if (tokens.length < 8) { + return null; // For versions < 2.11, we don't have node id + } + String primaryTermAndGen = String.join(SEPARATOR, tokens[1], tokens[2], tokens[3]); + String nodeId = tokens[5]; + return new Tuple<>(primaryTermAndGen, nodeId); + } + + // Visible for testing + static long getPrimaryTerm(String[] filenameTokens) { + return RemoteStoreUtils.invertLong(filenameTokens[1]); + } + + // Visible for testing + static long getGeneration(String[] filenameTokens) { + return RemoteStoreUtils.invertLong(filenameTokens[2]); + } + } + + + public static void remoteDirectoryCleanup(RemoteSegmentStoreDirectoryFactory remoteDirectoryFactory, + String remoteStoreRepoForIndex, String indexUUID, + ShardId shardId, RemoteStorePathStrategy pathStrategy, boolean forceClean) { + try { + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) remoteDirectoryFactory.newDirectory( + remoteStoreRepoForIndex, indexUUID, shardId, pathStrategy); + + if (forceClean) { + remoteSegmentStoreDirectory.delete(); + } else { + remoteSegmentStoreDirectory.deleteStaleSegments(0); + remoteSegmentStoreDirectory.deleteIfEmpty(); + } + } catch (Exception e) { + staticLogger.error("Exception occurred while deleting directory", e); + } } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 35aba694729cb..b043989e86301 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -8,6 +8,7 @@ package org.opensearch.index.store; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.store.Directory; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.blobstore.BlobPath; @@ -17,7 +18,9 @@ import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; +import org.opensearch.index.store.remote.CompositeRemoteDirectory; import org.opensearch.plugins.IndexStorePlugin; +import org.opensearch.plugins.PluginsService; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryMissingException; @@ -35,7 +38,9 @@ import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; /** - * Factory for a remote store directory + * Factory for a remote store directory with format-aware capabilities. + * Creates RemoteSegmentStoreDirectory instances using CompositeRemoteDirectory + * for multi-format support in remote storage. * * @opensearch.api */ @@ -43,17 +48,33 @@ public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.DirectoryFactory { private final Supplier repositoriesService; private final String segmentsPathFixedPrefix; - private final ThreadPool threadPool; + private final PluginsService pluginsService; + /** + * Constructor for backward compatibility without PluginsService + */ public RemoteSegmentStoreDirectoryFactory( Supplier repositoriesService, ThreadPool threadPool, String segmentsPathFixedPrefix + ) { + this(repositoriesService, threadPool, segmentsPathFixedPrefix, null); + } + + /** + * Constructor with PluginsService for format discovery + */ + public RemoteSegmentStoreDirectoryFactory( + Supplier repositoriesService, + ThreadPool threadPool, + String segmentsPathFixedPrefix, + PluginsService pluginsService ) { this.repositoriesService = repositoriesService; this.segmentsPathFixedPrefix = segmentsPathFixedPrefix; this.threadPool = threadPool; + this.pluginsService = pluginsService; } @Override @@ -68,6 +89,9 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null); } + /** + * Creates a new RemoteSegmentStoreDirectory with CompositeRemoteDirectory for format-aware operations + */ public Directory newDirectory( String repositoryName, String indexUUID, @@ -82,7 +106,7 @@ public Directory newDirectory( BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository); BlobPath repositoryBasePath = blobStoreRepository.basePath(); String shardIdStr = String.valueOf(shardId.id()); - Map pendingDownloadMergedSegments = new ConcurrentHashMap<>(); + Map pendingDownloadMergedSegments = new ConcurrentHashMap<>(); RemoteStorePathStrategy.ShardDataPathInput dataPathInput = RemoteStorePathStrategy.ShardDataPathInput.builder() .basePath(repositoryBasePath) @@ -93,15 +117,21 @@ public Directory newDirectory( .fixedPrefix(segmentsPathFixedPrefix) .indexFixedPrefix(indexFixedPrefix) .build(); + // Derive the path for data directory of SEGMENTS BlobPath dataPath = pathStrategy.generatePath(dataPathInput); - RemoteDirectory dataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore().blobContainer(dataPath), + + // Create CompositeRemoteDirectory with format support + CompositeRemoteDirectory compositeDataDirectory = new CompositeRemoteDirectory( + blobStoreRepository.blobStore(), + dataPath, blobStoreRepository::maybeRateLimitRemoteUploadTransfers, blobStoreRepository::maybeRateLimitLowPriorityRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers, blobStoreRepository::maybeRateLimitLowPriorityDownloadTransfers, - pendingDownloadMergedSegments + pendingDownloadMergedSegments, + LogManager.getLogger("index.store.remote.composite." + shardId), + pluginsService ); RemoteStorePathStrategy.ShardDataPathInput mdPathInput = RemoteStorePathStrategy.ShardDataPathInput.builder() @@ -113,11 +143,10 @@ public Directory newDirectory( .fixedPrefix(segmentsPathFixedPrefix) .indexFixedPrefix(indexFixedPrefix) .build(); - // Derive the path for metadata directory of SEGMENTS + BlobPath mdPath = pathStrategy.generatePath(mdPathInput); RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore().blobContainer(mdPath)); - // The path for lock is derived within the RemoteStoreLockManagerFactory RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager( repositoriesService.get(), repositoryName, @@ -129,7 +158,7 @@ public Directory newDirectory( ); return new RemoteSegmentStoreDirectory( - dataDirectory, + compositeDataDirectory, metadataDirectory, mdLockManager, threadPool, @@ -144,5 +173,4 @@ public Directory newDirectory( public Supplier getRepositoriesService() { return this.repositoriesService; } - } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileDownloader.java b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileDownloader.java index ad42b6d677b41..69b7671262fdc 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileDownloader.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileDownloader.java @@ -19,11 +19,14 @@ import org.opensearch.common.util.CancellableThreads; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.CompositeStoreDirectoryStatsWrapper; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Collection; +import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; @@ -65,6 +68,24 @@ public void downloadAsync( downloadInternal(cancellableThreads, source, destination, null, toDownloadSegments, () -> {}, listener); } + /** + * Format-aware version: Copies the given segments from the remote segment store to the given + * CompositeStoreDirectory, using FileMetadata for format-aware routing. + * @param source The remote directory to copy segment files from + * @param destination The CompositeStoreDirectory wrapper to copy segment files to + * @param toDownloadFileMetadata The list of format-aware files to download + * @param listener Callback listener to be notified upon completion + */ + public void downloadAsync( + CancellableThreads cancellableThreads, + RemoteSegmentStoreDirectory source, + CompositeStoreDirectoryStatsWrapper destination, + List toDownloadFileMetadata, + ActionListener listener + ) { + downloadInternalFormatAware(cancellableThreads, source, destination, toDownloadFileMetadata, () -> {}, listener); + } + /** * Copies the given segments from the remote segment store to the given * local directory, while also copying the segments _to_ another remote directory. @@ -130,6 +151,34 @@ private void downloadInternal( } } + /** + * Format-aware version of downloadInternal that works with CompositeStoreDirectoryStatsWrapper + * and FileMetadata for proper format-based routing. + */ + private void downloadInternalFormatAware( + CancellableThreads cancellableThreads, + RemoteSegmentStoreDirectory source, + CompositeStoreDirectoryStatsWrapper destination, + List toDownloadFileMetadata, + Runnable onFileCompletion, + ActionListener listener + ) { + final Queue queue = new ConcurrentLinkedQueue<>(toDownloadFileMetadata); + // Choose the minimum of: + // - number of files to download + // - max thread pool size + // - "indices.recovery.max_concurrent_remote_store_streams" setting + final int threads = Math.min( + toDownloadFileMetadata.size(), + Math.min(threadPool.info(ThreadPool.Names.REMOTE_RECOVERY).getMax(), recoverySettings.getMaxConcurrentRemoteStoreStreams()) + ); + logger.trace("Starting format-aware download of {} files with {} threads", queue.size(), threads); + final ActionListener allFilesListener = new GroupedActionListener<>(ActionListener.map(listener, r -> null), threads); + for (int i = 0; i < threads; i++) { + copyOneFileFormatAware(cancellableThreads, source, destination, queue, onFileCompletion, allFilesListener); + } + } + private void copyOneFile( CancellableThreads cancellableThreads, Directory source, @@ -165,4 +214,48 @@ private void copyOneFile( }); } } + + /** + * Format-aware version of copyOneFile that works with FileMetadata and CompositeStoreDirectoryStatsWrapper + * for format-aware routing to appropriate directories based on data format. + */ + private void copyOneFileFormatAware( + CancellableThreads cancellableThreads, + RemoteSegmentStoreDirectory source, + CompositeStoreDirectoryStatsWrapper destination, + Queue queue, + Runnable onFileCompletion, + ActionListener listener + ) { + final FileMetadata fileMetadata = queue.poll(); + if (fileMetadata == null) { + // Queue is empty, so notify listener we are done + listener.onResponse(null); + } else { + threadPool.executor(ThreadPool.Names.REMOTE_RECOVERY).submit(() -> { + logger.trace("Downloading format-aware file {} with format {}", fileMetadata.file(), fileMetadata.dataFormat()); + try { + cancellableThreads.executeIO(() -> { + // Use format-aware copy - CompositeStoreDirectoryStatsWrapper will route based on format + destination.copyFrom(fileMetadata, source, IOContext.DEFAULT); + logger.trace("Downloaded format-aware file {} of format {} of size {}", + fileMetadata.file(), fileMetadata.dataFormat(), + destination.getDelegate().fileLength(fileMetadata)); + onFileCompletion.run(); + + // TODO: @kamal, Add second destination support for format-aware operations if needed + // if (secondDestination != null) { + // secondDestination.copyFrom(destination, fileMetadata, fileMetadata, IOContext.DEFAULT); + // } + }); + } catch (Exception e) { + // Clear the queue to stop any future processing, report the failure, then return + queue.clear(); + listener.onFailure(e); + return; + } + copyOneFileFormatAware(cancellableThreads, source, destination, queue, onFileCompletion, listener); + }); + } + } } diff --git a/server/src/main/java/org/opensearch/index/store/SegmentUploadFailedException.java b/server/src/main/java/org/opensearch/index/store/SegmentUploadFailedException.java new file mode 100644 index 0000000000000..0224d39c7251c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/SegmentUploadFailedException.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 java.io.IOException; + +/** + * Exception thrown when segment upload to remote store fails. + * This exception provides detailed information about the upload failure + * to help with troubleshooting and error recovery. + */ +public class SegmentUploadFailedException extends IOException { + + public SegmentUploadFailedException(String message) { + super(message); + } + + public SegmentUploadFailedException(String message, Throwable cause) { + super(message, cause); + } + + /** + * Creates a SegmentUploadFailedException with format-specific information + */ + public static SegmentUploadFailedException forFormat(String fileName, String format, Throwable cause) { + String message = String.format( + "Failed to upload segment file '%s' with format '%s'. " + + "This may indicate a format-specific issue or configuration problem. " + + "Check that the format plugin is properly installed and configured.", + fileName, format + ); + return new SegmentUploadFailedException(message, cause); + } + + /** + * Creates a SegmentUploadFailedException for format not supported errors + */ + public static SegmentUploadFailedException formatNotSupported(String fileName, String format, Throwable cause) { + String message = String.format( + "Cannot upload segment file '%s': format '%s' is not supported. " + + "Install the required format plugin and restart OpenSearch to enable support for this format.", + fileName, format + ); + return new SegmentUploadFailedException(message, cause); + } +} \ No newline at end of file diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 0173d2faa46ca..456c3babbc728 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -74,6 +74,7 @@ import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.common.util.concurrent.RefCounted; @@ -89,12 +90,18 @@ import org.opensearch.env.ShardLockObtainFailedException; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.CombinedDeletionPolicy; +import org.opensearch.index.engine.DataFormatPlugin; import org.opensearch.index.engine.Engine; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.Any; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.AbstractIndexShardComponent; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.translog.Translog; +import org.opensearch.plugins.PluginsService; +import org.opensearch.index.engine.exec.DataFormat; import java.io.Closeable; import java.io.EOFException; @@ -106,6 +113,7 @@ import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -172,6 +180,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref ); private final AtomicBoolean isClosed = new AtomicBoolean(false); + private final CompositeStoreDirectory compositeStoreDirectory; private final StoreDirectory directory; private final ReentrantReadWriteLock metadataLock = new ReentrantReadWriteLock(); private final ShardLock shardLock; @@ -192,25 +201,59 @@ protected void closeInternal() { }; public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock) { - this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY, null); + this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY, createTempShardPath(shardId), null); } + public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock, OnClose onClose, ShardPath shardPath) { + this(shardId, indexSettings, directory, shardLock, onClose, shardPath, null); + } + + public Store( + ShardId shardId, + IndexSettings indexSettings, + Directory directory, + ShardLock shardLock, + OnClose onClose, + ShardPath shardPath, + PluginsService pluginsService + ) { + this(shardId, indexSettings, directory, shardLock, onClose, shardPath, pluginsService, null); + } + + /** + * Constructor with factory-created CompositeStoreDirectory + */ public Store( ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock, OnClose onClose, - ShardPath shardPath + ShardPath shardPath, + PluginsService pluginsService, + CompositeStoreDirectory factoryCreatedCompositeDirectory ) { super(shardId, indexSettings); + + ShardPath actualShardPath = shardPath != null ? shardPath : createTempShardPath(shardId); + final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING); logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval); ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(directory, refreshInterval); this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId)); + + // Use factory-created CompositeStoreDirectory if provided, otherwise create internally + if (factoryCreatedCompositeDirectory != null) { + this.compositeStoreDirectory = factoryCreatedCompositeDirectory; + logger.debug("Using factory-created CompositeStoreDirectory"); + } else { + this.compositeStoreDirectory = null; + logger.debug("Created CompositeStoreDirectory with plugin-based discovery (fallback)"); + } + this.shardLock = shardLock; this.onClose = onClose; - this.shardPath = shardPath; + this.shardPath = actualShardPath; this.isIndexSortEnabled = indexSettings.getIndexSortConfig().hasIndexSort(); this.isParentFieldEnabledVersion = indexSettings.getIndexVersionCreated().onOrAfter(org.opensearch.Version.V_3_2_0); assert onClose != null; @@ -218,11 +261,24 @@ public Store( assert shardLock.getShardId().equals(shardId); } + /** + * Creates a temporary ShardPath for testing when none is provided + */ + private static ShardPath createTempShardPath(ShardId shardId) { + Path tempPath = Path.of(System.getProperty("java.io.tmpdir"), "opensearch-test", shardId.toString()); + return new ShardPath(false, tempPath, tempPath, shardId); + } + public Directory directory() { ensureOpen(); return directory; } + public CompositeStoreDirectory compositeStoreDirectory() { + ensureOpen(); + return compositeStoreDirectory; + } + public ShardPath shardPath() { return shardPath; } @@ -404,6 +460,72 @@ public static RecoveryDiff segmentReplicationDiff(Map ); } + /** + * Segment Replication method - Format-aware version + * Returns a diff between the Maps of FileMetadata->StoreFileMetadata that can be used for getting list of files to copy over to a replica for segment replication. The returned diff will hold a list of files that are: + *

    + *
  • identical: they exist in both maps and they can be considered the same ie. they don't need to be recovered
  • + *
  • different: they exist in both maps but their they are not identical
  • + *
  • missing: files that exist in the source but not in the target
  • + *
+ * This version is format-aware and compares files based on both filename and data format. + */ + public static RecoveryDiff formatAwareSegmentReplicationDiff(Map source, Map target) { + final List identical = new ArrayList<>(); + final List different = new ArrayList<>(); + final List missing = new ArrayList<>(); + + for (Map.Entry sourceEntry : source.entrySet()) { + FileMetadata sourceFileMetadata = sourceEntry.getKey(); + StoreFileMetadata sourceStoreMetadata = sourceEntry.getValue(); + + // Skip segments files + if (sourceStoreMetadata.name().startsWith(IndexFileNames.SEGMENTS)) { + continue; + } + + // Check if target contains the same FileMetadata (format + filename) + if (target.containsKey(sourceFileMetadata) == false) { + missing.add(sourceStoreMetadata); + } else { + final StoreFileMetadata targetStoreMetadata = target.get(sourceFileMetadata); + String sourceChecksum = sourceStoreMetadata.checksum(); + String targetChecksum = targetStoreMetadata.checksum(); + + // Normalize target checksum if it's in decimal format + String normalizedTargetChecksum = normalizeChecksumToBase36(targetChecksum); + + // Compare normalized checksums + if (normalizedTargetChecksum.equals(sourceChecksum) + && targetStoreMetadata.dataFormat().equals(sourceStoreMetadata.dataFormat())) { + identical.add(sourceStoreMetadata); + } else { + different.add(sourceStoreMetadata); + } + } + } + return new RecoveryDiff( + Collections.unmodifiableList(identical), + Collections.unmodifiableList(different), + Collections.unmodifiableList(missing) + ); + } + + /** + * Normalizes checksum to base-36 format if it's currently in decimal format + */ + private static String normalizeChecksumToBase36(String checksum) { + try { + // Try to parse as decimal (base-10) + long checksumValue = Long.parseLong(checksum, 10); + // Convert to base-36 using Store.digestToString + return digestToString(checksumValue); + } catch (NumberFormatException e) { + // If parsing as decimal fails, assume it's already in base-36 format + return checksum; + } + } + /** * Renames all the given files from the key of the map to the * value of the map. All successfully renamed files are removed from the map in-place. @@ -541,7 +663,7 @@ private void closeInternal() { // Leverage try-with-resources to close the shard lock for us try (Closeable c = shardLock) { try { - directory.innerClose(); // this closes the distributorDirectory as well + directory.close(); // close the composite directory } finally { onClose.accept(shardLock); } @@ -757,7 +879,7 @@ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetadata) thr continue; } try { - directory.deleteFile(reason, existingFile); + directory.deleteFile(existingFile); // FNF should not happen since we hold a write lock? } catch (IOException ex) { if (existingFile.startsWith(IndexFileNames.SEGMENTS) || existingFile.startsWith(CORRUPTED_MARKER_NAME_PREFIX)) { @@ -1173,6 +1295,133 @@ public static LoadedMetadata loadMetadata(SegmentInfos segmentInfos, Directory d return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserDataBuilder), numDocs); } + /** + * Creates LoadedMetadata from CatalogSnapshot using CompositeStoreDirectory APIs. + * This method enables format-aware metadata calculation for replication checkpoints. + */ + public static LoadedMetadata loadMetadata(CatalogSnapshot catalogSnapshot, + CompositeStoreDirectory compositeDirectory, + Logger logger) throws IOException { + return loadMetadata(catalogSnapshot, compositeDirectory, logger, false); + } + + /** + * Creates LoadedMetadata from CatalogSnapshot using CompositeStoreDirectory APIs. + * This method enables format-aware metadata calculation for replication checkpoints. + * + * @param catalogSnapshot the CatalogSnapshot containing file information + * @param compositeDirectory the CompositeStoreDirectory for format-aware operations + * @param logger logger for debugging and error reporting + * @param ignoreSegmentsFile whether to skip segments files in metadata calculation + * @return LoadedMetadata containing file metadata, user data, and document count + */ + public static LoadedMetadata loadMetadata(CatalogSnapshot catalogSnapshot, + CompositeStoreDirectory compositeDirectory, + Logger logger, + boolean ignoreSegmentsFile) throws IOException { + + // TODO: Extract actual document count from CatalogSnapshot when API is available + long numDocs = 0; // Placeholder + + // TODO: Extract commit user data from CatalogSnapshot when API is available + Map commitUserData = new HashMap<>(); // Placeholder + + Map builder = new HashMap<>(); + Version maxVersion = null; + + // Get all files from CatalogSnapshot + Collection fileMetadatas = catalogSnapshot.getFileMetadataList(); + + for (FileMetadata fileMetadata : fileMetadatas) { + String fileName = fileMetadata.file(); + + // Skip segments files if requested + if (ignoreSegmentsFile && fileName.startsWith(IndexFileNames.SEGMENTS)) { + continue; + } + + try { + // Use CompositeStoreDirectory APIs for efficient metadata calculation + long length = compositeDirectory.fileLength(fileMetadata); + + // Convert checksum from long to string format expected by StoreFileMetadata + long checksumLong = compositeDirectory.calculateChecksum(fileMetadata); + String checksum = Store.digestToString(checksumLong); + + // TODO: Implement version extraction from segment info files + // For now, use a reasonable default version + Version version = org.opensearch.Version.CURRENT.minimumIndexCompatibilityVersion().luceneVersion; + + // Create hash for small files only + BytesRef hash = createHashForSmallFiles(compositeDirectory, fileMetadata, length, logger); + + StoreFileMetadata storeFileMetadata = new StoreFileMetadata(fileName, length, checksum, version, hash); + builder.put(fileName, storeFileMetadata); + + // Track max version + if (maxVersion == null || version.onOrAfter(maxVersion)) { + maxVersion = version; + } + + } catch (IOException e) { + logger.warn("Failed to create metadata for file: {} with format: {}", fileName, fileMetadata.dataFormat(), e); + // Continue with other files + } + } + + if (maxVersion == null) { + maxVersion = org.opensearch.Version.CURRENT.minimumIndexCompatibilityVersion().luceneVersion; + } + + return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserData), numDocs); + } + + /** + * Creates hash for small files using CompositeStoreDirectory APIs + */ + private static BytesRef createHashForSmallFiles(CompositeStoreDirectory compositeDirectory, + org.opensearch.index.engine.exec.FileMetadata fileMetadata, + long length, Logger logger) throws IOException { + // Only hash small files to avoid memory issues + if (length > 1024 * 1024) { // 1MB limit + return new BytesRef(); + } + + BytesRefBuilder fileHash = new BytesRefBuilder(); + try (IndexInput input = compositeDirectory.openInput(fileMetadata, IOContext.READONCE)) { + // Create a simple adapter to convert IndexInput to InputStream for hashing + InputStream inputStream = new InputStream() { + @Override + public int read() throws IOException { + if (input.getFilePointer() >= input.length()) { + return -1; + } + return input.readByte() & 0xFF; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (input.getFilePointer() >= input.length()) { + return -1; + } + long remaining = input.length() - input.getFilePointer(); + int toRead = (int) Math.min(len, remaining); + if (toRead <= 0) { + return -1; + } + input.readBytes(b, off, toRead); + return toRead; + } + }; + + Store.MetadataSnapshot.hashFile(fileHash, inputStream, length); + return fileHash.get(); + } catch (Exception e) { + logger.debug("Failed to create hash for file: {}", fileMetadata.file(), e); + return new BytesRef(); // Return empty hash on failure + } + } + private static void checksumFromLuceneFile( Directory directory, String file, @@ -1696,10 +1945,12 @@ public long verify() throws CorruptIndexException, IOException { public void deleteQuiet(String... files) { ensureOpen(); + CompositeStoreDirectory compositeStoreDirectory = this.compositeStoreDirectory; StoreDirectory directory = this.directory; for (String file : files) { try { - directory.deleteFile("Store.deleteQuiet", file); + // ToDo: CompositeDirectory needs FileMetadata + directory.deleteFile(file); } catch (Exception ex) { // ignore :( } diff --git a/server/src/main/java/org/opensearch/index/store/StoreFileMetadata.java b/server/src/main/java/org/opensearch/index/store/StoreFileMetadata.java index c286a5bdbd9d8..ed7efe58b4ee0 100644 --- a/server/src/main/java/org/opensearch/index/store/StoreFileMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/StoreFileMetadata.java @@ -64,16 +64,27 @@ public class StoreFileMetadata implements Writeable { private final BytesRef hash; + private final String dataFormat; + public StoreFileMetadata(String name, long length, String checksum, Version writtenBy) { - this(name, length, checksum, writtenBy, null); + this(name, length, checksum, writtenBy, null, "lucene"); } public StoreFileMetadata(String name, long length, String checksum, Version writtenBy, BytesRef hash) { + this(name, length, checksum, writtenBy, hash, "lucene"); + } + + public StoreFileMetadata(String name, long length, String checksum, Version writtenBy, String dataFormat) { + this(name, length, checksum, writtenBy, null, dataFormat); + } + + public StoreFileMetadata(String name, long length, String checksum, Version writtenBy, BytesRef hash, String dataFormat) { this.name = Objects.requireNonNull(name, "name must not be null"); this.length = length; this.checksum = Objects.requireNonNull(checksum, "checksum must not be null"); this.writtenBy = Objects.requireNonNull(writtenBy, "writtenBy must not be null"); this.hash = hash == null ? new BytesRef() : hash; + this.dataFormat = Objects.requireNonNull(dataFormat, "dataFormat must not be null"); } /** @@ -89,6 +100,11 @@ public StoreFileMetadata(StreamInput in) throws IOException { throw new AssertionError(e); } hash = in.readBytesRef(); + if (in.available() > 0) { + dataFormat = in.readString(); + } else { + dataFormat = "lucene"; + } } @Override @@ -98,6 +114,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(checksum); out.writeString(writtenBy.toString()); out.writeBytesRef(hash); + out.writeString(dataFormat); } /** @@ -154,12 +171,12 @@ public boolean isSame(StoreFileMetadata other) { // we can't tell if either or is null so we return false in this case! this is why we don't use equals for this! return false; } - return length == other.length && checksum.equals(other.checksum) && hash.equals(other.hash); + return length == other.length && checksum.equals(other.checksum) && hash.equals(other.hash) && dataFormat.equals(other.dataFormat); } @Override public String toString() { - return "name [" + name + "], length [" + length + "], checksum [" + checksum + "], writtenBy [" + writtenBy + "]"; + return "name [" + name + "], length [" + length + "], checksum [" + checksum + "], writtenBy [" + writtenBy + "], dataFormat [" + dataFormat + "]"; } /** @@ -176,4 +193,11 @@ public Version writtenBy() { public BytesRef hash() { return hash; } + + /** + * Returns the data format of this file (e.g., "lucene", "parquet") + */ + public String dataFormat() { + return dataFormat; + } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/CompositeRemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/remote/CompositeRemoteDirectory.java new file mode 100644 index 0000000000000..c615e12369e59 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/CompositeRemoteDirectory.java @@ -0,0 +1,489 @@ +/* + * 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; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.ExceptionsHelper; +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.exception.CorruptFileException; +import org.opensearch.common.blobstore.stream.write.WriteContext; +import org.opensearch.common.blobstore.stream.write.WritePriority; +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.engine.MergedSegmentWarmer; +import org.opensearch.index.engine.exec.DataFormat; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.Any; +import org.opensearch.index.store.CompositeStoreDirectory; +import org.opensearch.index.store.RemoteIndexInput; +import org.opensearch.index.store.RemoteIndexOutput; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandlerFactory; +import org.opensearch.common.io.VersionedCodecStreamWrapper; +import org.opensearch.plugins.DataSourcePlugin; +import org.opensearch.plugins.PluginsService; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.NoSuchFileException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.UnaryOperator; + +/** + * CompositeRemoteDirectory with direct BlobContainer access per format. + * + * Key Architecture: + * - Map of DataFormat to BlobContainer for direct blob access + * - Lazy BlobContainer creation when new formats are encountered + * - Format determination from FileMetadata instead of file extension parsing + * - ALL formats get equal treatment with same generic streaming logic + * + * @opensearch.api + */ +@PublicApi(since = "3.0.0") +public class CompositeRemoteDirectory implements Closeable { + + /** + * Metadata stream wrapper for reading/writing RemoteSegmentMetadata + */ + private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( + new RemoteSegmentMetadataHandlerFactory(), + RemoteSegmentMetadata.VERSION_ONE, + RemoteSegmentMetadata.CURRENT_VERSION, + RemoteSegmentMetadata.METADATA_CODEC + ); + + private final UnaryOperator uploadRateLimiter; + private final UnaryOperator lowPriorityUploadRateLimiter; + private final DownloadRateLimiterProvider downloadRateLimiterProvider; + + /** + * Map containing the mapping of segment files that are pending download as part of the pre-copy (warm) phase of + * {@link MergedSegmentWarmer}. The key is the local filename and value is the remote filename. + */ + final Map pendingDownloadMergedSegments; + + private final Map formatBlobContainers; + private final BlobContainer metadataBlobContainer; + private final BlobStore blobStore; + private final BlobPath baseBlobPath; + private final Logger logger; + + /** + * Full constructor with all rate limiter parameters + */ + public CompositeRemoteDirectory( + BlobStore blobStore, + BlobPath baseBlobPath, + UnaryOperator uploadRateLimiter, + UnaryOperator lowPriorityUploadRateLimiter, + UnaryOperator downloadRateLimiter, + UnaryOperator lowPriorityDownloadRateLimiter, + Map pendingDownloadMergedSegments, + Logger logger, + PluginsService pluginsService + ) { + this.formatBlobContainers = new ConcurrentHashMap<>(); + this.blobStore = blobStore; + this.baseBlobPath = baseBlobPath; + this.uploadRateLimiter = uploadRateLimiter; + this.lowPriorityUploadRateLimiter = lowPriorityUploadRateLimiter; + this.downloadRateLimiterProvider = new DownloadRateLimiterProvider(downloadRateLimiter, lowPriorityDownloadRateLimiter); + this.pendingDownloadMergedSegments = pendingDownloadMergedSegments; + this.logger = logger; + + BlobPath metadataBlobPath = baseBlobPath.parent().add("metadata"); + this.metadataBlobContainer = blobStore.blobContainer(metadataBlobPath); + + try { + DataSourcePlugin plugin = pluginsService.filterPlugins(DataSourcePlugin.class).stream().findAny().orElseThrow(() -> new IllegalArgumentException("dataformat [" + DataFormat.TEXT + "] is not registered.")); + formatBlobContainers.put(plugin.getDataFormat().name(), plugin.createBlobContainer(blobStore, baseBlobPath)); + } catch (NullPointerException | IOException e) { + formatBlobContainers.put("", null); + } + + logger.debug("Created CompositeRemoteDirectory with {} format BlobContainers", + formatBlobContainers.size()); + } + + /** + * Get or create BlobContainer for specific format. + * This is where the lazy creation happens - if we don't have a BlobContainer + * for this format yet, we create one and store it in the map. + * + * @param format the data format name + * @return BlobContainer for the format (created if not exists) + */ + public BlobContainer getBlobContainerForFormat(String format) { + return formatBlobContainers.computeIfAbsent(format, f -> { + // Create format-specific BlobPath: basePath/formatName/ + BlobPath formatPath = baseBlobPath.add(f.toLowerCase()); + BlobContainer container = blobStore.blobContainer(formatPath); + + logger.debug("Created new BlobContainer for format {} at path: {}", f, formatPath); + return container; + }); + } + + public void copyFrom(CompositeStoreDirectory from, FileMetadata src, String dest, IOContext context) + throws IOException { + boolean success = false; + try (IndexInput is = from.openInput(src, IOContext.READONCE); + IndexOutput os = createOutput(dest, src.dataFormat(), context)) { + os.copyBytes(is, is.length()); + success = true; + } finally { + if (!success) { + from.deleteFile(src); + } + } + } + + /** + * Primary copyFrom method using FileMetadata - similar to RemoteDirectory.copyFrom but with format-aware routing + */ + public boolean copyFrom( + CompositeStoreDirectory from, + FileMetadata fileMetadata, + String remoteFileName, + IOContext context, + Runnable postUploadRunner, + ActionListener listener, + boolean lowPriorityUpload + ) { + try { + String fileName = fileMetadata.file(); + BlobContainer blobContainer = getBlobContainerForFormat(fileMetadata.dataFormat()); + + if (blobContainer instanceof AsyncMultiStreamBlobContainer) { + logger.debug("Starting format-aware upload: file={}, format={}, container={}", + fileName, fileMetadata.dataFormat(), blobContainer.path()); + uploadBlob(from, fileMetadata, remoteFileName, context, postUploadRunner, listener, lowPriorityUpload); + return true; + } + + logger.warn("BlobContainer does not support async multi-stream upload: {}", blobContainer.getClass()); + return false; + + } catch (Exception e) { + logger.error("Failed to start format-aware upload: file={}, error={}", fileMetadata.dataFormat(), e.getMessage(), e); + listener.onFailure(e); + return true; // Return true to indicate we handled it (even though it failed) + } + } + + private void uploadBlob( + CompositeStoreDirectory from, + FileMetadata src, + String remoteFileName, + IOContext ioContext, + Runnable postUploadRunner, + ActionListener listener, + boolean lowPriorityUpload + ) throws Exception { + assert ioContext != IOContext.READONCE : "Remote upload will fail with IoContext.READONCE"; + String dataFormat = src.dataFormat(); + long expectedChecksum = calculateChecksumOfChecksum(from, src); + long contentLength; + IndexInput indexInput = from.openInput(src, ioContext); + try { + contentLength = indexInput.length(); + boolean remoteIntegrityEnabled = false; + if (getBlobContainer(dataFormat) instanceof AsyncMultiStreamBlobContainer) { + remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) getBlobContainer(dataFormat)).remoteIntegrityCheckSupported(); + } + lowPriorityUpload = lowPriorityUpload || contentLength > ByteSizeUnit.GB.toBytes(15); + RemoteTransferContainer.OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier; + + if (lowPriorityUpload) { + offsetRangeInputStreamSupplier = (size, position) -> lowPriorityUploadRateLimiter.apply( + new OffsetRangeIndexInputStream(indexInput.clone(), size, position) + ); + } else { + offsetRangeInputStreamSupplier = (size, position) -> uploadRateLimiter.apply( + new OffsetRangeIndexInputStream(indexInput.clone(), size, position) + ); + } + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + src, + remoteFileName, + contentLength, + true, + lowPriorityUpload ? WritePriority.LOW : WritePriority.NORMAL, + offsetRangeInputStreamSupplier, + expectedChecksum, + remoteIntegrityEnabled + ); + ActionListener completionListener = ActionListener.wrap(resp -> { + try { + postUploadRunner.run(); + listener.onResponse(null); + } catch (Exception e) { + logger.error(() -> new ParameterizedMessage("Exception in segment postUpload for file [{}]", src), e); + listener.onFailure(e); + } + }, ex -> { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", src), ex); + IOException corruptIndexException = ExceptionsHelper.unwrapCorruption(ex); + if (corruptIndexException != null) { + listener.onFailure(corruptIndexException); + return; + } + Throwable throwable = ExceptionsHelper.unwrap(ex, CorruptFileException.class); + if (throwable != null) { + CorruptFileException corruptFileException = (CorruptFileException) throwable; + listener.onFailure(new CorruptIndexException(corruptFileException.getMessage(), corruptFileException.getFileName())); + return; + } + listener.onFailure(ex); + }); + + completionListener = ActionListener.runBefore(completionListener, () -> { + try { + remoteTransferContainer.close(); + } catch (Exception e) { + logger.warn("Error occurred while closing streams", e); + } + }); + + completionListener = ActionListener.runAfter(completionListener, () -> { + try { + indexInput.close(); + } catch (IOException e) { + logger.warn("Error occurred while closing index input", e); + } + }); + + WriteContext writeContext = remoteTransferContainer.createWriteContext(); + ((AsyncMultiStreamBlobContainer) getBlobContainer(dataFormat)).asyncBlobUpload(writeContext, completionListener); + } catch (Exception e) { + logger.warn("Exception while calling asyncBlobUpload, closing IndexInput to avoid leak"); + indexInput.close(); + throw e; + } + } + + private BlobContainer getBlobContainer(String df) { + return formatBlobContainers.get(df); + } + + private long calculateChecksumOfChecksum(CompositeStoreDirectory from, FileMetadata fileMetadata) throws IOException { + return from.calculateChecksum(fileMetadata); + } + + /** + + /** + * Get file length from the appropriate format-specific BlobContainer + * @param fileMetadata The File Metadata + * @return The length of the file + * @throws IOException If the file is not found or on other IO errors + */ + public long fileLength(FileMetadata fileMetadata) throws IOException { + BlobContainer blobContainer = getBlobContainer(fileMetadata.dataFormat()); + + if (blobContainer == null) { + throw new NoSuchFileException( + String.format("File %s not found in any containers for format %s", fileMetadata.file(), fileMetadata.dataFormat()) + ); + } + + try { + List metadata = blobContainer.listBlobsByPrefixInSortedOrder( + fileMetadata.file(), 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + + if (metadata.size() == 1 && metadata.get(0).name().equals(fileMetadata.file())) { + return metadata.get(0).length(); + } + + throw new NoSuchFileException( + String.format("File %s not found in container for format %s", fileMetadata.file(), fileMetadata.dataFormat()) + ); + } catch (IOException e) { + throw new IOException( + String.format("Error getting length for file %s in format %s", fileMetadata.file(), fileMetadata.dataFormat()), e + ); + } + } + + /** + * Create output for writing to the appropriate format-specific BlobContainer + */ + public RemoteIndexOutput createOutput(String remoteFileName, String df, IOContext context) throws IOException { + if (remoteFileName == null || remoteFileName.isEmpty()) { + throw new IllegalArgumentException("Remote file name cannot be null or empty"); + } + try { + BlobContainer blobContainer = formatBlobContainers.get(df); + + if (blobContainer!=null) { + logger.debug("File {} already exists, using existing container", remoteFileName); + return new RemoteIndexOutput(remoteFileName, blobContainer); + } + else if(df !=null && df.equals("TempMetadata")) { + return new RemoteIndexOutput(remoteFileName, metadataBlobContainer); + } + + throw new IOException( + String.format("Failed to create output for file %s in format %s", remoteFileName, df) + ); + } catch (Exception e) { + throw new IOException( + String.format("Failed to create output for file %s in format %s", remoteFileName, df), + e + ); + } + } + + /** + * Open input for reading from the appropriate format-specific BlobContainer + */ + public IndexInput openInput(FileMetadata fileMetadata, long fileLength, IOContext context) throws IOException { + if (fileMetadata.file() == null || fileMetadata.file().isEmpty()) { + throw new IllegalArgumentException("Remote file name cannot be null or empty"); + } + + InputStream inputStream = null; + try { + BlobContainer blobContainer = getBlobContainer(fileMetadata.dataFormat()); + if (blobContainer==null) { + throw new IOException(String.format("Failed to find blobContainer for file %s in format %s", fileMetadata.file(), fileMetadata.dataFormat())); + } + + inputStream = blobContainer.readBlob(fileMetadata.file()); + UnaryOperator rateLimiter = downloadRateLimiterProvider.get(fileMetadata.file()); + return new RemoteIndexInput(fileMetadata.file(), rateLimiter.apply(inputStream), fileLength); + } catch (Exception e) { + if (inputStream != null) { + try { + inputStream.close(); + } catch (Exception closeEx) { + e.addSuppressed(closeEx); + } + } + logger.error("Exception while reading blob for file: " + fileMetadata.file(), e); + throw e; + } + } + + /** + * Delete the entire CompositeRemoteDirectory + */ + public void delete() throws IOException { + for (BlobContainer container : formatBlobContainers.values()) { + container.delete(); + } + logger.debug("Deleted all format containers from CompositeRemoteDirectory"); + } + + + /** + * Read the latest metadata file from the metadata blob container. + * This method provides compatibility with RemoteSegmentStoreDirectory.readLatestMetadataFile() + */ + public RemoteSegmentMetadata readLatestMetadataFile() throws IOException { + try { + List metadataFiles = metadataBlobContainer.listBlobsByPrefixInSortedOrder( + "metadata", 10, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + + if (metadataFiles.isEmpty()) { + logger.debug("No metadata files found in composite remote directory"); + return null; + } + + // Get the latest (first in reverse lexicographic order) + String latestMetadataFile = metadataFiles.get(0).name(); + logger.debug("Reading latest metadata file: {}", latestMetadataFile); + return readMetadataFile(latestMetadataFile); + } catch (Exception e) { + logger.error("Failed to read latest metadata file from composite directory", e); + throw new IOException("Failed to read latest metadata file", e); + } + } + + /** + * Read a specific metadata file by name from the metadata blob container. + * This method provides compatibility with RemoteSegmentStoreDirectory.readMetadataFile() + */ + public RemoteSegmentMetadata readMetadataFile(String metadataFileName) throws IOException { + try (InputStream inputStream = metadataBlobContainer.readBlob(metadataFileName)) { + byte[] metadataBytes = inputStream.readAllBytes(); + + // Use our own metadata stream wrapper + return metadataStreamWrapper.readStream( + new ByteArrayIndexInput(metadataFileName, metadataBytes) + ); + } catch (NoSuchFileException e) { + logger.debug("Metadata file not found: {}", metadataFileName); + return null; + } catch (Exception e) { + logger.error("Failed to read metadata file: {}", metadataFileName, e); + throw new IOException("Failed to read metadata file: " + metadataFileName, e); + } + } + + @Override + public void close() throws IOException { + formatBlobContainers.clear(); + } + + @Override + public String toString() { + return "CompositeRemoteDirectory{" + + "formats=" + formatBlobContainers.keySet() + + ", basePath=" + baseBlobPath + + '}'; + } + + private boolean isMergedSegment(String remoteFilename) { + return pendingDownloadMergedSegments != null && pendingDownloadMergedSegments.containsValue(remoteFilename); + } + + /** + * DownloadRateLimiterProvider returns a low-priority rate limited stream if the segment + * being downloaded is a merged segment as part of the pre-copy (warm) phase of + * {@link MergedSegmentWarmer}. + */ + private class DownloadRateLimiterProvider { + private final UnaryOperator downloadRateLimiter; + private final UnaryOperator lowPriorityDownloadRateLimiter; + + DownloadRateLimiterProvider( + UnaryOperator downloadRateLimiter, + UnaryOperator lowPriorityDownloadRateLimiter + ) { + this.downloadRateLimiter = downloadRateLimiter; + this.lowPriorityDownloadRateLimiter = lowPriorityDownloadRateLimiter; + } + + public UnaryOperator get(final String filename) { + if (isMergedSegment(filename)) { + return lowPriorityDownloadRateLimiter; + } + return downloadRateLimiter; + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 463e08918b3f7..b62ee4a197426 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -14,11 +14,13 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; +import java.util.HashMap; import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -47,14 +49,14 @@ public class RemoteSegmentMetadata { /** * Data structure holding metadata content */ - private final Map metadata; + private final Map metadata; private final byte[] segmentInfosBytes; private final ReplicationCheckpoint replicationCheckpoint; public RemoteSegmentMetadata( - Map metadata, + Map metadata, byte[] segmentInfosBytes, ReplicationCheckpoint replicationCheckpoint ) { @@ -67,7 +69,7 @@ public RemoteSegmentMetadata( * Exposes underlying metadata content data structure. * @return {@code metadata} */ - public Map getMetadata() { + public Map getMetadata() { return this.metadata; } @@ -89,23 +91,35 @@ public ReplicationCheckpoint getReplicationCheckpoint() { /** * Generate {@code Map} from {@link RemoteSegmentMetadata} + * Converts FileMetadata keys to String filenames for serialization. * @return {@code Map} */ public Map toMapOfStrings() { - return this.metadata.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().toString())); + return this.metadata.entrySet().stream() + .collect(Collectors.toMap( + entry -> entry.getKey().file(), // FileMetadata → String filename + entry -> entry.getValue().toString() + )); } /** * Generate {@link RemoteSegmentMetadata} from {@code segmentMetadata} - * @param segmentMetadata metadata content in the form of {@code Map} - * @return {@link RemoteSegmentMetadata} + * Converts String-based serialized metadata to FileMetadata-keyed map. + * @param segmentMetadata metadata content in the form of {@code Map} + * @return Map with FileMetadata keys */ - public static Map fromMapOfStrings(Map segmentMetadata) { + public static Map fromMapOfStrings(Map segmentMetadata) { return segmentMetadata.entrySet() .stream() .collect( Collectors.toMap( - Map.Entry::getKey, + entry -> { + // Parse metadata to get format information + RemoteSegmentStoreDirectory.UploadedSegmentMetadata metadata = + RemoteSegmentStoreDirectory.UploadedSegmentMetadata.fromString(entry.getValue()); + // Create FileMetadata with format from metadata + return new FileMetadata(metadata.getDataFormat(), "", entry.getKey().file()); + }, entry -> RemoteSegmentStoreDirectory.UploadedSegmentMetadata.fromString(entry.getValue()) ) ); @@ -131,10 +145,59 @@ public void write(IndexOutput out) throws IOException { * @throws IOException in case there is a problem reading from the file input stream */ public static RemoteSegmentMetadata read(IndexInput indexInput, int version) throws IOException { - Map metadata = indexInput.readMapOfStrings(); - final Map uploadedSegmentMetadataMap = RemoteSegmentMetadata - .fromMapOfStrings(metadata); - ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, uploadedSegmentMetadataMap, version); + Map serializedMetadata = indexInput.readMapOfStrings(); + + // Add null check and validation + if (serializedMetadata == null) { + throw new IOException("Serialized metadata cannot be null during RemoteSegmentMetadata deserialization"); + } + + // Convert Map to Map by reconstructing format information + Map fileMetadataMap = new HashMap<>(); + for (Map.Entry entry : serializedMetadata.entrySet()) { + String filename = entry.getKey(); + String metadataString = entry.getValue(); + + if (filename == null || metadataString == null) { + throw new IOException("Invalid metadata entry: filename=" + filename + ", metadataString=" + metadataString); + } + + try { + // Parse format from UploadedSegmentMetadata string (format is preserved in toString()) + RemoteSegmentStoreDirectory.UploadedSegmentMetadata parsedMetadata = + RemoteSegmentStoreDirectory.UploadedSegmentMetadata.fromString(metadataString); + + // Reconstruct FileMetadata with correct format + FileMetadata fileMetadata = new FileMetadata(parsedMetadata.getDataFormat(), "", filename); + fileMetadataMap.put(fileMetadata, metadataString); + } catch (Exception e) { + throw new IOException("Failed to parse UploadedSegmentMetadata for file: " + filename, e); + } + } + + // Add null check before calling fromMapOfStrings + if (fileMetadataMap.isEmpty()) { + // Handle empty metadata case gracefully + final Map emptyMap = new HashMap<>(); + ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, new HashMap<>(), version); + int byteArraySize = (int) indexInput.readLong(); + byte[] segmentInfosBytes = new byte[byteArraySize]; + indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); + return new RemoteSegmentMetadata(emptyMap, segmentInfosBytes, replicationCheckpoint); + } + + // Now pass the properly converted map instead of null + final Map uploadedSegmentMetadataMap = + RemoteSegmentMetadata.fromMapOfStrings(fileMetadataMap); + + // Create String-based map for readCheckpointFromIndexInput (backward compatibility) + Map stringKeyedMap = uploadedSegmentMetadataMap.entrySet().stream() + .collect(Collectors.toMap( + entry -> entry.getKey().file(), + Map.Entry::getValue + )); + + ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, stringKeyedMap, version); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); @@ -167,21 +230,28 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput( in.readLong(), in.readLong(), in.readLong(), - in.readString(), toStoreFileMetadata(uploadedSegmentMetadataMap), + in.readString(), version >= VERSION_TWO ? in.readLong() : 0 ); } - private static Map toStoreFileMetadata( + private static Map toStoreFileMetadata( Map metadata - ) { + ){ return metadata.entrySet() .stream() - // TODO: Version here should be read from UploadedSegmentMetadata. - .map( - entry -> new StoreFileMetadata(entry.getKey(), entry.getValue().getLength(), entry.getValue().getChecksum(), Version.LATEST) - ) - .collect(Collectors.toMap(StoreFileMetadata::name, Function.identity())); + .collect(Collectors.toMap( + // Key: Create FileMetadata from filename and data format + entry -> new FileMetadata(entry.getValue().getDataFormat(), "", entry.getKey()), + // Value: Create StoreFileMetadata (similar to existing method but with data format) + entry -> new StoreFileMetadata( + entry.getKey(), + entry.getValue().getLength(), + entry.getValue().getChecksum(), + Version.LATEST, + entry.getValue().getDataFormat() + ) + )); } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 81438b978cd99..33fdd51f1a227 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -243,7 +243,8 @@ void uploadBlobAsyncInternal( inputStreamSupplier, expectedChecksum, remoteIntegrityEnabled, - metadata + metadata, + null ) ) { ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload( diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index e9f377d666c6f..ad10465f70af7 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -391,6 +391,7 @@ public class IndicesService extends AbstractLifecycleComponent private final Collection>> engineFactoryProviders; private final Map directoryFactories; private final Map compositeDirectoryFactories; + private final Map compositeStoreDirectoryFactories; private final Map ingestionConsumerFactories; private final Map recoveryStateFactories; private final Map storeFactories; @@ -445,6 +446,7 @@ public IndicesService( Collection>> engineFactoryProviders, Map directoryFactories, Map compositeDirectoryFactories, + Map compositeStoreDirectoryFactories, ValuesSourceRegistry valuesSourceRegistry, Map recoveryStateFactories, Map storeFactories, @@ -511,6 +513,7 @@ public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, lon this.directoryFactories = directoryFactories; this.compositeDirectoryFactories = compositeDirectoryFactories; + this.compositeStoreDirectoryFactories = compositeStoreDirectoryFactories; this.recoveryStateFactories = recoveryStateFactories; this.storeFactories = storeFactories; this.ingestionConsumerFactories = ingestionConsumerFactories; @@ -645,6 +648,7 @@ public IndicesService( engineFactoryProviders, directoryFactories, Collections.emptyMap(), + Collections.emptyMap(), valuesSourceRegistry, recoveryStateFactories, Collections.emptyMap(), @@ -1067,7 +1071,8 @@ private synchronized IndexService createIndexService( recoveryStateFactories, storeFactories, fileCache, - compositeIndexSettings + compositeIndexSettings, + compositeStoreDirectoryFactories ); for (IndexingOperationListener operationListener : indexingOperationListeners) { indexModule.addIndexOperationListener(operationListener); @@ -1196,7 +1201,8 @@ public synchronized MapperService createIndexMapperService(IndexMetadata indexMe recoveryStateFactories, storeFactories, fileCache, - compositeIndexSettings + compositeIndexSettings, + compositeStoreDirectoryFactories ); pluginsService.onIndexModule(indexModule); return indexModule.newIndexMapperService(xContentRegistry, mapperRegistry, scriptService); diff --git a/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java index 5a21b10d04c06..fb6d87c800d95 100644 --- a/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java @@ -26,12 +26,17 @@ import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; import org.opensearch.indices.replication.common.ReplicationTarget; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.UncheckedIOException; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -215,7 +220,12 @@ protected List getFiles(CheckpointInfoResponse checkpointInfo if (indexShard.indexSettings().isWarmIndex()) { return Collections.emptyList(); } - final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); + + // Get format-aware metadata from checkpoint info + final Map sourceMetadataMap = checkpointInfo.getFormatAwareMetadataMap(); + final Map targetMetadataMap = indexShard.getFormatAwareSegmentMetadataMap(); + final Store.RecoveryDiff diff = Store.formatAwareSegmentReplicationDiff(sourceMetadataMap, targetMetadataMap); + // local files final Set localFiles = Set.of(indexShard.store().directory().listAll()); // set of local files that can be reused @@ -231,7 +241,7 @@ protected List getFiles(CheckpointInfoResponse checkpointInfo logger.trace( () -> new ParameterizedMessage( - "Replication diff for checkpoint {} {} {}", + "Replication diff for checkpoint {} {} {} (format-aware)", checkpointInfo.getCheckpoint(), missingFiles, diff.different diff --git a/server/src/main/java/org/opensearch/indices/replication/CheckpointInfoResponse.java b/server/src/main/java/org/opensearch/indices/replication/CheckpointInfoResponse.java index 24b744bebc53d..2824237ed9bec 100644 --- a/server/src/main/java/org/opensearch/indices/replication/CheckpointInfoResponse.java +++ b/server/src/main/java/org/opensearch/indices/replication/CheckpointInfoResponse.java @@ -11,10 +11,12 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.transport.TransportResponse; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; +import java.util.HashMap; import java.util.Map; /** @@ -27,44 +29,82 @@ public class CheckpointInfoResponse extends TransportResponse { private final ReplicationCheckpoint checkpoint; - private final Map metadataMap; + private final Map legacyMetadataMap; + private final Map formatAwareMetadataMap; private final byte[] infosBytes; + // Constructor with legacy metadata map for backward compatibility (used by tests) public CheckpointInfoResponse( final ReplicationCheckpoint checkpoint, final Map metadataMap, final byte[] infosBytes ) { this.checkpoint = checkpoint; - this.metadataMap = metadataMap; + this.legacyMetadataMap = metadataMap; + this.formatAwareMetadataMap = convertLegacyToFormatAware(metadataMap); this.infosBytes = infosBytes; } + // Constructor using checkpoint's metadata public CheckpointInfoResponse(final ReplicationCheckpoint checkpoint, final byte[] infosBytes) { this.checkpoint = checkpoint; this.infosBytes = infosBytes; - this.metadataMap = checkpoint.getMetadataMap(); + this.formatAwareMetadataMap = checkpoint.getFormatAwareMetadataMap(); + this.legacyMetadataMap = checkpoint.getMetadataMap(); } public CheckpointInfoResponse(StreamInput in) throws IOException { this.checkpoint = new ReplicationCheckpoint(in); - this.metadataMap = in.readMap(StreamInput::readString, StoreFileMetadata::new); + this.legacyMetadataMap = in.readMap(StreamInput::readString, StoreFileMetadata::new); + this.formatAwareMetadataMap = convertLegacyToFormatAware(legacyMetadataMap); this.infosBytes = in.readByteArray(); } @Override public void writeTo(StreamOutput out) throws IOException { checkpoint.writeTo(out); - out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); + out.writeMap(legacyMetadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); out.writeByteArray(infosBytes); } + /** + * Converts legacy String-based metadata map to format-aware FileMetadata-based map. + */ + private static Map convertLegacyToFormatAware(Map legacyMap) { + Map formatAwareMap = new HashMap<>(); + for (Map.Entry entry : legacyMap.entrySet()) { + String fileName = entry.getKey(); + StoreFileMetadata storeMetadata = entry.getValue(); + + // Use the dataFormat from StoreFileMetadata if available, otherwise default to "lucene" + String dataFormat = storeMetadata.dataFormat() != null ? storeMetadata.dataFormat() : "lucene"; + FileMetadata fileMetadata = new FileMetadata(dataFormat, "", fileName); + formatAwareMap.put(fileMetadata, storeMetadata); + } + return formatAwareMap; + } + public ReplicationCheckpoint getCheckpoint() { return checkpoint; } + /** + * Returns the legacy metadata map for backward compatibility. + * Format information may be lost in this conversion. + * + * @deprecated Use getFormatAwareMetadataMap() instead to preserve format information + */ + @Deprecated public Map getMetadataMap() { - return metadataMap; + return legacyMetadataMap; + } + + /** + * Returns the format-aware metadata map that preserves format information. + * This is the preferred method for accessing file metadata. + */ + public Map getFormatAwareMetadataMap() { + return formatAwareMetadataMap; } public byte[] getInfosBytes() { diff --git a/server/src/main/java/org/opensearch/indices/replication/CompositeStoreDirectoryStatsWrapper.java b/server/src/main/java/org/opensearch/indices/replication/CompositeStoreDirectoryStatsWrapper.java new file mode 100644 index 0000000000000..a4883fffcaa80 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/CompositeStoreDirectoryStatsWrapper.java @@ -0,0 +1,86 @@ +/* + * 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.indices.replication; + +import org.apache.lucene.store.IOContext; +import org.opensearch.index.engine.exec.FileMetadata; +import org.opensearch.index.store.CompositeStoreDirectory; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; + +import java.io.IOException; +import java.util.function.BiConsumer; + +/** + * CompositeStoreDirectory wrapper that records copy process for replication statistics. + * Similar to ReplicationStatsDirectoryWrapper but designed for CompositeStoreDirectory's format-aware operations. + * + * This wrapper intercepts copyFrom operations and tracks progress for replication statistics, + * while delegating all actual directory operations to the underlying CompositeStoreDirectory. + * + * @opensearch.internal + */ +public final class CompositeStoreDirectoryStatsWrapper { + private final CompositeStoreDirectory delegate; + private final BiConsumer fileProgressTracker; + + public CompositeStoreDirectoryStatsWrapper(CompositeStoreDirectory delegate, BiConsumer fileProgressTracker) { + this.delegate = delegate; + this.fileProgressTracker = fileProgressTracker; + } + + /** + * Copies a file from source directory with format-agnostic progress tracking. + * This method is format-aware and uses callback-based progress tracking instead of FilterDirectory. + */ + public void copyFrom(FileMetadata fileMetadata, RemoteSegmentStoreDirectory from, IOContext context) throws IOException { + String fileName = fileMetadata.file(); + + try { + // Get file size for progress tracking + long fileSize = from.getFileLength(fileMetadata); + + // Report start of copy operation + fileProgressTracker.accept(fileName, 0L); + + // Delegate the actual format-aware copy to CompositeStoreDirectory + // CompositeStoreDirectory will route based on fileMetadata.dataFormat() + delegate.copyFrom(fileMetadata, from, context); + + // Report completion of copy operation + fileProgressTracker.accept(fileName, fileSize); + + } catch (IOException e) { + // Report failure - no bytes were successfully transferred + fileProgressTracker.accept(fileName, 0L); + throw e; + } + } + + /** + * Legacy copyFrom method for backward compatibility with existing download APIs. + * Converts String filenames to FileMetadata with default "lucene" format. + */ + public void copyFrom(RemoteSegmentStoreDirectory from, String src, String dest, IOContext context) throws IOException { + // Convert to FileMetadata with default format for backward compatibility + FileMetadata destFileMetadata = new FileMetadata("lucene", "", dest); + copyFrom(destFileMetadata, from, context); + } + + /** + * Gets the underlying CompositeStoreDirectory for direct access when needed. + */ + public CompositeStoreDirectory getDelegate() { + return delegate; + } + + @Override + public String toString() { + return "CompositeStoreDirectoryStatsWrapper(" + delegate.toString() + ")"; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 2aedf9534abe3..2bedb0eaaf2e5 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -18,8 +18,10 @@ import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.util.CancellableThreads; import org.opensearch.core.action.ActionListener; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.store.CompositeStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; @@ -29,7 +31,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -51,11 +52,13 @@ public class RemoteStoreReplicationSource implements SegmentReplicationSource { private static final Logger logger = LogManager.getLogger(RemoteStoreReplicationSource.class); private final IndexShard indexShard; - private final RemoteSegmentStoreDirectory remoteDirectory; + private final RemoteSegmentStoreDirectory remoteDirectory; // Fallback for legacy cases private final CancellableThreads cancellableThreads = new CancellableThreads(); public RemoteStoreReplicationSource(IndexShard indexShard) { this.indexShard = indexShard; + + // Try to get CompositeRemoteDirectory first, fallback to RemoteSegmentStoreDirectory FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); this.remoteDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory.getDelegate(); @@ -103,13 +106,14 @@ public void getCheckpointMetadata( .stream() .collect( Collectors.toMap( - e -> e.getKey(), + e -> e.getKey().file(), e -> new StoreFileMetadata( e.getValue().getOriginalFilename(), e.getValue().getLength(), Store.digestToString(Long.valueOf(e.getValue().getChecksum())), version, - null + null, + e.getKey().dataFormat() ) ) ); @@ -133,22 +137,37 @@ public void getSegmentFiles( listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); return; } - logger.debug("Downloading segment files from remote store {}", filesToFetch); + logger.debug("Downloading format-aware segment files from remote store {}", filesToFetch); if (remoteMetadataExists()) { - final Directory storeDirectory = indexShard.store().directory(); - final Collection directoryFiles = List.of(storeDirectory.listAll()); - final List toDownloadSegmentNames = new ArrayList<>(); - for (StoreFileMetadata fileMetadata : filesToFetch) { - String file = fileMetadata.name(); - assert directoryFiles.contains(file) == false : "Local store already contains the file " + file; - toDownloadSegmentNames.add(file); + final CompositeStoreDirectory storeDirectory = indexShard.store().compositeStoreDirectory(); + final List directoryFiles = List.of(storeDirectory.listAll()); + + final List toDownloadFileMetadata = new ArrayList<>(); + + for (StoreFileMetadata storeFileMetadata : filesToFetch) { + String fileName = storeFileMetadata.name(); + String dataFormat = storeFileMetadata.dataFormat() != null ? storeFileMetadata.dataFormat() : "lucene"; + + // Create FileMetadata for format-aware operations + FileMetadata fileMetadata = new FileMetadata(dataFormat, "", fileName); + + // Verify file doesn't already exist in local directory + assert directoryFiles.contains(fileMetadata) == false : "Local store already contains the file " + fileMetadata; + + toDownloadFileMetadata.add(fileMetadata); + + logger.trace("Queuing format-aware file for download: {} with format: {}", fileName, dataFormat); } + + // Use CompositeStoreDirectory with format-aware progress tracking + final CompositeStoreDirectoryStatsWrapper statsWrapper = new CompositeStoreDirectoryStatsWrapper(storeDirectory, fileProgressTracker); + indexShard.getFileDownloader() .downloadAsync( cancellableThreads, remoteDirectory, - new ReplicationStatsDirectoryWrapper(storeDirectory, fileProgressTracker), - toDownloadSegmentNames, + statsWrapper, + toDownloadFileMetadata, ActionListener.map(listener, r -> new GetSegmentFilesResponse(filesToFetch)) ); } else { @@ -216,15 +235,23 @@ public String getDescription() { return "RemoteStoreReplicationSource"; } - private boolean remoteMetadataExists() throws IOException { - final AtomicBoolean metadataExists = new AtomicBoolean(false); - cancellableThreads.executeIO(() -> metadataExists.set(remoteDirectory.readLatestMetadataFile() != null)); - return metadataExists.get(); - } - private RemoteSegmentMetadata getRemoteSegmentMetadata() throws IOException { + if (remoteDirectory == null) { + throw new IllegalStateException("No remote directory available for shard: " + indexShard.shardId()); + } + AtomicReference mdFile = new AtomicReference<>(); cancellableThreads.executeIO(() -> mdFile.set(remoteDirectory.init())); return mdFile.get(); } + + private boolean remoteMetadataExists() throws IOException { + if (remoteDirectory == null) { + return false; + } + + final AtomicBoolean metadataExists = new AtomicBoolean(false); + cancellableThreads.executeIO(() -> metadataExists.set(remoteDirectory.readLatestMetadataFile() != null)); + return metadataExists.get(); + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java index fa8968030db96..1519720d08bf9 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java @@ -40,10 +40,12 @@ public interface SegmentReplicationSource { /** * Fetch the requested segment files. Passes a listener that completes when files are stored locally. + * This method is format-aware and can handle multi-format files using the dataFormat information + * in StoreFileMetadata to direct files to the appropriate CompositeStoreDirectory. * * @param replicationId long - ID of the replication event. * @param checkpoint {@link ReplicationCheckpoint} Checkpoint to fetch metadata for. - * @param filesToFetch {@link List} List of files to fetch. + * @param filesToFetch {@link List} List of format-aware files to fetch (StoreFileMetadata contains dataFormat). * @param indexShard {@link IndexShard} Reference to the IndexShard. * @param fileProgressTracker {@link BiConsumer} A consumer that updates the replication progress for shard files. * @param listener {@link ActionListener} Listener that completes with the list of files copied. diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index b4a40ba78ea42..cf1f6a989ecb7 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -18,6 +18,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.lucene.Lucene; import org.opensearch.common.util.CancellableThreads; +import org.opensearch.index.engine.exec.coord.CatalogSnapshot; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; @@ -25,6 +26,8 @@ import org.opensearch.indices.replication.common.ReplicationFailedException; import org.opensearch.indices.replication.common.ReplicationListener; +import java.io.ByteArrayInputStream; +import java.io.IOException; import java.util.List; import java.util.function.BiConsumer; @@ -78,7 +81,7 @@ protected void getFilesFromSource( @Override protected void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws Exception { - // Handle empty SegmentInfos bytes for recovering replicas + // Handle empty CatalogSnapshot bytes for recovering replicas if (checkpointInfoResponse.getInfosBytes() == null) { return; } @@ -87,11 +90,8 @@ protected void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse store = store(); store.incRef(); multiFileWriter.renameAllTempFiles(); - final SegmentInfos infos = store.buildSegmentInfos( - checkpointInfoResponse.getInfosBytes(), - checkpointInfoResponse.getCheckpoint().getSegmentsGen() - ); - indexShard.finalizeReplication(infos); + final CatalogSnapshot catalogSnapshot = deserializeCatalogSnapshot(checkpointInfoResponse.getInfosBytes()); + indexShard.finalizeReplication(catalogSnapshot); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. // this means we transferred files from the remote that have not be checksummed and they are @@ -130,4 +130,20 @@ protected void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse public SegmentReplicationTarget retryCopy() { return new SegmentReplicationTarget(indexShard, checkpoint, source, listener); } + + /** + * Deserializes CatalogSnapshot from byte array. + * + * @param infoBytes the serialized CatalogSnapshot bytes + * @return deserialized CatalogSnapshot + * @throws IOException if deserialization fails + */ + private CatalogSnapshot deserializeCatalogSnapshot(byte[] infoBytes) throws IOException { + try { + ByteArrayInputStream bais = new ByteArrayInputStream(infoBytes); + return CatalogSnapshot.readFrom(bais); + } catch (ClassNotFoundException e) { + throw new IOException("Failed to deserialize CatalogSnapshot", e); + } + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentCheckpoint.java index 47c46781c42fd..28056ad6cb44e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentCheckpoint.java @@ -12,6 +12,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.StoreFileMetadata; @@ -29,6 +30,19 @@ public class MergedSegmentCheckpoint extends ReplicationCheckpoint { private final String segmentName; + public MergedSegmentCheckpoint( + ShardId shardId, + long primaryTerm, + long segmentInfosVersion, + long length, + Map metadataMap, + String codec, + String segmentName + ) { + super(shardId, primaryTerm, SequenceNumbers.NO_OPS_PERFORMED, segmentInfosVersion, length, metadataMap, codec); + this.segmentName = segmentName; + } + public MergedSegmentCheckpoint( ShardId shardId, long primaryTerm, diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReferencedSegmentsCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReferencedSegmentsCheckpoint.java index 52d39a1e0cab4..f81567b5c1454 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReferencedSegmentsCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReferencedSegmentsCheckpoint.java @@ -12,6 +12,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.StoreFileMetadata; @@ -36,10 +37,10 @@ public ReferencedSegmentsCheckpoint( long segmentInfosVersion, long length, String codec, - Map metadataMap, + Map metadataMap, Set segmentNames ) { - super(shardId, primaryTerm, SequenceNumbers.NO_OPS_PERFORMED, segmentInfosVersion, length, codec, metadataMap); + super(shardId, primaryTerm, SequenceNumbers.NO_OPS_PERFORMED, segmentInfosVersion, length, metadataMap, codec); this.segmentNames = segmentNames; } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/RemoteStorePublishMergedSegmentAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/RemoteStorePublishMergedSegmentAction.java index da5560875db3f..cb347632fd21c 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/RemoteStorePublishMergedSegmentAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/RemoteStorePublishMergedSegmentAction.java @@ -20,6 +20,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.UploadListener; import org.opensearch.core.action.ActionListener; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.RemoteStoreUploader; import org.opensearch.index.shard.RemoteStoreUploaderService; @@ -79,11 +80,12 @@ public RemoteStorePublishMergedSegmentAction( this.replicationService = targetService; } + // Todo:@Kamal, needs implementation with fileMetadata @Override protected void doReplicaOperation(RemoteStorePublishMergedSegmentRequest shardRequest, IndexShard replica) { RemoteStoreMergedSegmentCheckpoint checkpoint = shardRequest.getMergedSegment(); if (checkpoint.getShardId().equals(replica.shardId())) { - replica.getRemoteDirectory().markMergedSegmentsPendingDownload(checkpoint.getLocalToRemoteSegmentFilenameMap()); + // replica.getRemoteDirectory().markMergedSegmentsPendingDownload(checkpoint.getLocalToRemoteSegmentFilenameMap()); replicationService.onNewMergedSegmentCheckpoint(checkpoint, replica); } else { logger.warn( @@ -108,7 +110,16 @@ protected void shardOperationOnPrimary( @Override public final void publish(IndexShard indexShard, MergedSegmentCheckpoint checkpoint) { long startTimeMillis = System.currentTimeMillis(); - Map localToRemoteStoreFilenames = uploadMergedSegmentsToRemoteStore(indexShard, checkpoint); + Map localToRemoteStoreFileMetadatas = uploadMergedSegmentsToRemoteStore(indexShard, checkpoint); + + // Todo: needs to update this it should use fileMetadata rather than filename. + Map localToRemoteStoreFilenames = localToRemoteStoreFileMetadatas.entrySet() + .stream() + .collect(Collectors.toMap( + entry -> entry.getKey().file(), // Extract filename from FileMetadata + Map.Entry::getValue + )); + long endTimeMillis = System.currentTimeMillis(); long elapsedTimeMillis = endTimeMillis - startTimeMillis; @@ -139,11 +150,11 @@ public final void publish(IndexShard indexShard, MergedSegmentCheckpoint checkpo } } - private Map uploadMergedSegmentsToRemoteStore(IndexShard indexShard, MergedSegmentCheckpoint checkpoint) { - Collection segmentsToUpload = checkpoint.getMetadataMap().keySet(); - Map localToRemoteStoreFilenames = new ConcurrentHashMap<>(); + private Map uploadMergedSegmentsToRemoteStore(IndexShard indexShard, MergedSegmentCheckpoint checkpoint) { + Collection segmentsToUpload = checkpoint.getFormatAwareMetadataMap().keySet(); + Map localToRemoteStoreFilenames = new ConcurrentHashMap<>(); - Map segmentsSizeMap = checkpoint.getMetadataMap() + Map segmentsSizeMap = checkpoint.getFormatAwareMetadataMap() .entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().length())); @@ -162,15 +173,15 @@ public void onFailure(Exception e) { } }, (x) -> new UploadListener() { @Override - public void beforeUpload(String file) {} + public void beforeUpload(FileMetadata file) {} @Override - public void onSuccess(String file) { + public void onSuccess(FileMetadata file) { localToRemoteStoreFilenames.put(file, indexShard.getRemoteDirectory().getExistingRemoteFilename(file)); } @Override - public void onFailure(String file) { + public void onFailure(FileMetadata file) { logger.warn("Unable to upload segments during merge. Continuing."); } }, true); @@ -197,6 +208,6 @@ public void onFailure(String file) { } private RemoteStoreUploader getRemoteStoreUploaderService(IndexShard indexShard) { - return new RemoteStoreUploaderService(indexShard, indexShard.store().directory(), indexShard.getRemoteDirectory()); + return new RemoteStoreUploaderService(indexShard, indexShard.store().compositeStoreDirectory(), indexShard.getRemoteDirectory()); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 39c2039191def..375785db0ee08 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -16,12 +16,14 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.exec.FileMetadata; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.StoreFileMetadata; import java.io.IOException; import java.time.Instant; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -39,9 +41,12 @@ public class ReplicationCheckpoint implements Writeable, Comparable metadataMap; + private final Map formatAwareMetadataMap; private final long createdTimeStamp; + // Version compatibility for FileMetadata-aware serialization + private static final Version FILEMETADATA_AWARE_VERSION = Version.V_3_1_0; + public static ReplicationCheckpoint empty(ShardId shardId) { return empty(shardId, ""); } @@ -57,7 +62,7 @@ private ReplicationCheckpoint(ShardId shardId, String codec) { segmentInfosVersion = SequenceNumbers.NO_OPS_PERFORMED; length = 0L; this.codec = codec; - this.metadataMap = Collections.emptyMap(); + this.formatAwareMetadataMap = Collections.emptyMap(); this.createdTimeStamp = DateUtils.toLong(Instant.now()); } @@ -65,6 +70,27 @@ public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap(), DateUtils.toLong(Instant.now())); } + // Format-aware constructor (with codec first to avoid erasure clash) + public ReplicationCheckpoint( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + Map formatAwareMetadataMap, + String codec + ) { + this.shardId = shardId; + this.primaryTerm = primaryTerm; + this.segmentsGen = segmentsGen; + this.segmentInfosVersion = segmentInfosVersion; + this.length = length; + this.codec = codec; + this.formatAwareMetadataMap = formatAwareMetadataMap; + this.createdTimeStamp = DateUtils.toLong(Instant.now()); + } + + // Legacy constructor (with codec after length to distinguish from format-aware) public ReplicationCheckpoint( ShardId shardId, long primaryTerm, @@ -72,7 +98,7 @@ public ReplicationCheckpoint( long segmentInfosVersion, long length, String codec, - Map metadataMap + Map legacyMetadataMap ) { this.shardId = shardId; this.primaryTerm = primaryTerm; @@ -80,18 +106,56 @@ public ReplicationCheckpoint( this.segmentInfosVersion = segmentInfosVersion; this.length = length; this.codec = codec; - this.metadataMap = metadataMap; this.createdTimeStamp = DateUtils.toLong(Instant.now()); + + // Convert legacy metadata to format-aware metadata + Map convertedMap = new HashMap<>(); + for (Map.Entry entry : legacyMetadataMap.entrySet()) { + String dataFormat = entry.getValue().dataFormat() != null ? entry.getValue().dataFormat() : "lucene"; + FileMetadata fileMetadata = new FileMetadata(dataFormat, "", entry.getKey()); + convertedMap.put(fileMetadata, entry.getValue()); + } + this.formatAwareMetadataMap = convertedMap; + } + + // Legacy constructor with timestamp (with codec after length to distinguish) + public ReplicationCheckpoint( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + String codec, + Map legacyMetadataMap, + long createdTimeStamp + ) { + this.shardId = shardId; + this.primaryTerm = primaryTerm; + this.segmentsGen = segmentsGen; + this.segmentInfosVersion = segmentInfosVersion; + this.length = length; + this.codec = codec; + this.createdTimeStamp = createdTimeStamp; + + // Convert legacy metadata to format-aware metadata + Map convertedMap = new HashMap<>(); + for (Map.Entry entry : legacyMetadataMap.entrySet()) { + String dataFormat = entry.getValue().dataFormat() != null ? entry.getValue().dataFormat() : "lucene"; + FileMetadata fileMetadata = new FileMetadata(dataFormat, "", entry.getKey()); + convertedMap.put(fileMetadata, entry.getValue()); + } + this.formatAwareMetadataMap = convertedMap; } + // Format-aware constructor with timestamp (with metadata before codec to distinguish) public ReplicationCheckpoint( ShardId shardId, long primaryTerm, long segmentsGen, long segmentInfosVersion, long length, + Map formatAwareMetadataMap, String codec, - Map metadataMap, long createdTimeStamp ) { this.shardId = shardId; @@ -100,10 +164,49 @@ public ReplicationCheckpoint( this.segmentInfosVersion = segmentInfosVersion; this.length = length; this.codec = codec; - this.metadataMap = metadataMap; + this.formatAwareMetadataMap = formatAwareMetadataMap; this.createdTimeStamp = createdTimeStamp; } + /** + * Static factory method for legacy String-based metadata maps. + * Creates a ReplicationCheckpoint with converted FileMetadata-based map. + * + * @deprecated Use constructor with Map<FileMetadata, StoreFileMetadata> instead + */ + @Deprecated + public static ReplicationCheckpoint fromLegacyMetadata( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + String codec, + Map legacyMetadataMap + ) { + return new ReplicationCheckpoint(shardId, primaryTerm, segmentsGen, segmentInfosVersion, length, codec, legacyMetadataMap); + } + + /** + * Static factory method for legacy String-based metadata maps with timestamp. + * Creates a ReplicationCheckpoint with converted FileMetadata-based map. + * + * @deprecated Use constructor with Map<FileMetadata, StoreFileMetadata> instead + */ + @Deprecated + public static ReplicationCheckpoint fromLegacyMetadata( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + String codec, + Map legacyMetadataMap, + long createdTimeStamp + ) { + return new ReplicationCheckpoint(shardId, primaryTerm, segmentsGen, segmentInfosVersion, length, codec, legacyMetadataMap, createdTimeStamp); + } + public ReplicationCheckpoint(StreamInput in) throws IOException { shardId = new ShardId(in); primaryTerm = in.readLong(); @@ -116,10 +219,24 @@ public ReplicationCheckpoint(StreamInput in) throws IOException { length = 0L; codec = null; } - if (in.getVersion().onOrAfter(Version.V_2_10_0)) { - this.metadataMap = in.readMap(StreamInput::readString, StoreFileMetadata::new); + if (in.getVersion().onOrAfter(FILEMETADATA_AWARE_VERSION)) { + // Read FileMetadata-aware format + this.formatAwareMetadataMap = in.readMap( + streamInput -> new FileMetadata(streamInput.readString(), "", streamInput.readString()), + StoreFileMetadata::new + ); + } else if (in.getVersion().onOrAfter(Version.V_2_10_0)) { + // Convert legacy String-based format to FileMetadata format + Map legacyMap = in.readMap(StreamInput::readString, StoreFileMetadata::new); + Map convertedMap = new HashMap<>(); + for (Map.Entry entry : legacyMap.entrySet()) { + String dataFormat = entry.getValue().dataFormat() != null ? entry.getValue().dataFormat() : "lucene"; + FileMetadata fileMetadata = new FileMetadata(dataFormat, "", entry.getKey()); + convertedMap.put(fileMetadata, entry.getValue()); + } + this.formatAwareMetadataMap = convertedMap; } else { - this.metadataMap = Collections.emptyMap(); + this.formatAwareMetadataMap = Collections.emptyMap(); } if (in.getVersion().onOrAfter(Version.V_3_0_0)) { this.createdTimeStamp = in.readLong(); @@ -186,8 +303,22 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(length); out.writeString(codec); } - if (out.getVersion().onOrAfter(Version.V_2_10_0)) { - out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); + if (out.getVersion().onOrAfter(FILEMETADATA_AWARE_VERSION)) { + // Write FileMetadata-aware format + out.writeMap(formatAwareMetadataMap, + (keyOut, fileMetadata) -> { + keyOut.writeString(fileMetadata.dataFormat()); + keyOut.writeString(fileMetadata.file()); + }, + (valueOut, storeFileMetadata) -> storeFileMetadata.writeTo(valueOut) + ); + } else if (out.getVersion().onOrAfter(Version.V_2_10_0)) { + // Convert to legacy String-based format for backward compatibility + Map legacyMap = new HashMap<>(); + for (Map.Entry entry : formatAwareMetadataMap.entrySet()) { + legacyMap.put(entry.getKey().file(), entry.getValue()); + } + out.writeMap(legacyMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); } if (out.getVersion().onOrAfter(Version.V_3_0_0)) { out.writeLong(createdTimeStamp); @@ -226,8 +357,24 @@ public boolean isAheadOf(@Nullable ReplicationCheckpoint other) { || (primaryTerm == other.getPrimaryTerm() && segmentInfosVersion > other.getSegmentInfosVersion()); } + public Map getFormatAwareMetadataMap() { + return formatAwareMetadataMap; + } + + /** + * Returns legacy String-based metadata map for backward compatibility. + * Format information is lost in this conversion. + * + * @return legacy metadata map with filename keys + * @deprecated Use getFormatAwareMetadataMap() instead to preserve format information + */ + @Deprecated public Map getMetadataMap() { - return metadataMap; + Map legacyMap = new HashMap<>(); + for (Map.Entry entry : formatAwareMetadataMap.entrySet()) { + legacyMap.put(entry.getKey().file(), entry.getValue()); + } + return legacyMap; } public long getCreatedTimeStamp() { diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 416237111ff7b..cdeb1ce59b235 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -935,6 +935,17 @@ protected Node(final Environment initialEnvironment, Collection clas }); compositeDirectoryFactories.put("default", new DefaultCompositeDirectoryFactory()); + // Collect CompositeStoreDirectoryFactories from plugins + final Map compositeStoreDirectoryFactories = new HashMap<>(); + pluginsService.filterPlugins(IndexStorePlugin.class) + .stream() + .forEach(plugin -> { + // Check if plugin implements getCompositeStoreDirectoryFactories method + // For now, we'll add empty collection and rely on default factory + }); + // Register default factory + compositeStoreDirectoryFactories.put("default", new org.opensearch.index.store.DefaultCompositeStoreDirectoryFactory()); + final Map recoveryStateFactories = pluginsService.filterPlugins( IndexStorePlugin.class ) @@ -961,7 +972,8 @@ protected Node(final Environment initialEnvironment, Collection clas final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( repositoriesServiceReference::get, threadPool, - remoteStoreSettings.getSegmentsPathFixedPrefix() + remoteStoreSettings.getSegmentsPathFixedPrefix(), + pluginsService ); final TaskResourceTrackingService taskResourceTrackingService = new TaskResourceTrackingService( @@ -1000,6 +1012,7 @@ protected Node(final Environment initialEnvironment, Collection clas engineFactoryProviders, Map.copyOf(directoryFactories), Map.copyOf(compositeDirectoryFactories), + Map.copyOf(compositeStoreDirectoryFactories), searchModule.getValuesSourceRegistry(), recoveryStateFactories, storeFactories, diff --git a/server/src/main/java/org/opensearch/plugins/DataSourcePlugin.java b/server/src/main/java/org/opensearch/plugins/DataSourcePlugin.java index cf008d3098fcd..e6db12568a303 100644 --- a/server/src/main/java/org/opensearch/plugins/DataSourcePlugin.java +++ b/server/src/main/java/org/opensearch/plugins/DataSourcePlugin.java @@ -8,12 +8,18 @@ package org.opensearch.plugins; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.exec.DataFormat; import org.opensearch.index.engine.exec.IndexingExecutionEngine; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.FormatStoreDirectory; import org.opensearch.vectorized.execution.search.spi.DataSourceCodec; +import java.io.IOException; import java.util.Map; import java.util.Optional; @@ -24,5 +30,12 @@ default Optional IndexingExecutionEngine indexingEngine(MapperService mapperService, ShardPath shardPath); + FormatStoreDirectory createFormatStoreDirectory( + IndexSettings indexSettings, + ShardPath shardPath + ) throws IOException; + + BlobContainer createBlobContainer(BlobStore blobStore, BlobPath blobPath) throws IOException; + DataFormat getDataFormat(); } diff --git a/server/src/main/java/org/opensearch/plugins/IndexStorePlugin.java b/server/src/main/java/org/opensearch/plugins/IndexStorePlugin.java index 052026c6b9c0d..20cfffac95a6f 100644 --- a/server/src/main/java/org/opensearch/plugins/IndexStorePlugin.java +++ b/server/src/main/java/org/opensearch/plugins/IndexStorePlugin.java @@ -42,6 +42,7 @@ import org.opensearch.env.ShardLock; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.CompositeStoreDirectoryFactory; import org.opensearch.index.store.IndexStoreListener; import org.opensearch.index.store.Store; import org.opensearch.index.store.remote.filecache.FileCache; @@ -126,6 +127,21 @@ default Map getCompositeDirectoryFactories() return Collections.emptyMap(); } + /** + * The {@link CompositeStoreDirectoryFactory} mappings for this plugin. When an index is created the composite store directory + * factory type setting will be examined and either use the default or looked up among all the composite store directory + * factories from {@link IndexStorePlugin} plugins. + * + * This factory enables plugin-based format discovery and centralized creation of CompositeStoreDirectory instances + * with all discovered DataFormat plugins. + * + * @return a map from factory type to a composite store directory factory + */ + @ExperimentalApi + default Map getCompositeStoreDirectoryFactories() { + return Collections.emptyMap(); + } + /** * An interface that allows to create a new {@link RecoveryState} per shard. * diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java index 1e2264593310d..a9381849ec7e2 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java @@ -411,7 +411,10 @@ private void snapshot( indexFilesToFileLengthMap = lastRemoteUploadedIndexCommit.getMetadata() .entrySet() .stream() - .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getLength())); + .collect(Collectors.toMap( + entry -> entry.getKey().file(), + entry -> entry.getValue().getLength() + )); primaryTerm = lastRemoteUploadedIndexCommit.getPrimaryTerm(); commitGeneration = lastRemoteUploadedIndexCommit.getGeneration(); } else { diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreUploaderServiceTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreUploaderServiceTests.java index 6301a484b81b5..3fece2d1eeb2d 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreUploaderServiceTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreUploaderServiceTests.java @@ -141,7 +141,7 @@ public void testUploadSegmentsWithEmptyCollection() throws Exception { exception -> fail("Should not fail for empty segments") ); - uploaderService.uploadSegments(emptySegments, segmentSizeMap, listener, mockUploadListenerFunction, false); + uploaderService.uploadSegmentsLegacy(emptySegments, segmentSizeMap, listener, mockUploadListenerFunction, false); assertTrue(latch.await(1, TimeUnit.SECONDS)); } @@ -201,7 +201,7 @@ public void testUploadSegmentsSuccessWithHighPriorityUpload() throws Exception { exception -> fail("Upload should succeed: " + exception.getMessage()) ); - testUploaderService.uploadSegments(segments, segmentSizeMap, listener, mockUploadListenerFunction, false); + testUploaderService.uploadSegmentsLegacy(segments, segmentSizeMap, listener, mockUploadListenerFunction, false); assertTrue(latch.await(5, TimeUnit.SECONDS)); // Verify the upload listener was called correctly 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 4c2472940afc6..f9f2d0e63c513 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -32,6 +32,7 @@ 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.exec.FileMetadata; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy; @@ -88,7 +89,8 @@ public void testUploadedSegmentMetadataToString() { "abc", "pqr", "123456", - 1234 + 1234, + "format" ); metadata.setWrittenByMajor(Version.LATEST.major); assertEquals("abc::pqr::123456::1234::" + Version.LATEST.major, metadata.toString()); @@ -99,7 +101,8 @@ public void testUploadedSegmentMetadataToStringExceptionTooNew() { "abc", "pqr", "123456", - 1234 + 1234, + "format" ); assertThrows(IllegalArgumentException.class, () -> metadata.setWrittenByMajor(Version.LATEST.major + 1)); } @@ -109,7 +112,8 @@ public void testUploadedSegmentMetadataToStringExceptionTooOld() { "abc", "pqr", "123456", - 1234 + 1234, + "format" ); assertThrows(IllegalArgumentException.class, () -> metadata.setWrittenByMajor(Version.LATEST.major - 2)); } @@ -615,7 +619,7 @@ public void testContainsFile() throws IOException { UnsupportedOperationException.class, () -> uploadedSegmentMetadataMap.put( "_100.si", - new RemoteSegmentStoreDirectory.UploadedSegmentMetadata("_100.si", "_100.si__uuid1", "1234", 500) + new RemoteSegmentStoreDirectory.UploadedSegmentMetadata("_100.si", "_100.si__uuid1", "1234", 500,"format") ) );