diff --git a/.idea/runConfigurations/Debug_OpenSearch.xml b/.idea/runConfigurations/Debug_OpenSearch.xml
deleted file mode 100644
index c18046f873477..0000000000000
--- a/.idea/runConfigurations/Debug_OpenSearch.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
\ No newline at end of file
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 2baa708bbb6eb..a4a476808b80d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -55,6 +55,21 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
- Add streaming cardinality aggregator ([#19484](https://github.com/opensearch-project/OpenSearch/pull/19484))
- Disable request cache for streaming aggregation queries ([#19520](https://github.com/opensearch-project/OpenSearch/pull/19520))
+- Add support for a ForkJoinPool type ([#19008](https://github.com/opensearch-project/OpenSearch/pull/19008))
+- Add seperate shard limit validation for local and remote indices ([#19532](https://github.com/opensearch-project/OpenSearch/pull/19532))
+- Use Lucene `pack` method for `half_float` and `usigned_long` when using `ApproximatePointRangeQuery`.
+- Add a mapper for context aware segments grouping criteria ([#19233](https://github.com/opensearch-project/OpenSearch/pull/19233))
+- Return full error for GRPC error response ([#19568](https://github.com/opensearch-project/OpenSearch/pull/19568))
+- Add support for repository with Server side encryption enabled and client side encryption as well based on a flag. ([#19630)](https://github.com/opensearch-project/OpenSearch/pull/19630))
+- Add pluggable gRPC interceptors with explicit ordering([#19005](https://github.com/opensearch-project/OpenSearch/pull/19005))
+- Add BindableServices extension point to transport-grpc-spi ([#19304](https://github.com/opensearch-project/OpenSearch/pull/19304))
+- Add metrics for the merged segment warmer feature ([#18929](https://github.com/opensearch-project/OpenSearch/pull/18929))
+- Add pointer based lag metric in pull-based ingestion ([#19635](https://github.com/opensearch-project/OpenSearch/pull/19635))
+- Introduced internal API for retrieving metadata about requested indices from transport actions ([#18523](https://github.com/opensearch-project/OpenSearch/pull/18523))
+- Add cluster defaults for merge autoThrottle, maxMergeThreads, and maxMergeCount; Add segment size filter to the merged segment warmer ([#19629](https://github.com/opensearch-project/OpenSearch/pull/19629))
+- Add build-tooling to run in FIPS environment ([#18921](https://github.com/opensearch-project/OpenSearch/pull/18921))
+- Add SMILE/CBOR/YAML document format support to Bulk GRPC endpoint ([#19744](https://github.com/opensearch-project/OpenSearch/pull/19744))
+
### Changed
- Refactor `if-else` chains to use `Java 17 pattern matching switch expressions`(([#18965](https://github.com/opensearch-project/OpenSearch/pull/18965))
- Add CompletionStage variants to methods in the Client Interface and default to ActionListener impl ([#18998](https://github.com/opensearch-project/OpenSearch/pull/18998))
diff --git a/gradle/run.gradle b/gradle/run.gradle
index 3b5a3eebab756..a2d96d31ad096 100644
--- a/gradle/run.gradle
+++ b/gradle/run.gradle
@@ -40,6 +40,34 @@ testClusters {
testDistribution = 'archive'
if (numZones > 1) numberOfZones = numZones
if (numNodes > 1) numberOfNodes = numNodes
+ // S3 repository configuration
+ if (findProperty("enableS3")) {
+ plugin(':plugins:repository-s3')
+ if (findProperty("s3Endpoint")) {
+ setting 's3.client.default.endpoint', findProperty("s3Endpoint")
+ }
+ setting 's3.client.default.region', findProperty("s3Region") ?: 'us-east-1'
+ keystore 's3.client.default.access_key', findProperty("s3AccessKey") ?: System.getenv("AWS_ACCESS_KEY_ID") ?: 'test'
+ keystore 's3.client.default.secret_key', findProperty("s3SecretKey") ?: System.getenv("AWS_SECRET_ACCESS_KEY") ?: 'test'
+
+
+ // Remote store configuration
+ setting 'node.attr.remote_store.segment.repository', 'my-s3-repo'
+ setting 'node.attr.remote_store.translog.repository', 'my-s3-repo'
+ setting 'node.attr.remote_store.state.repository', 'my-s3-repo'
+ setting 'cluster.remote_store.state.enabled', 'true'
+ setting 'node.attr.remote_store.repository.my-s3-repo.type', 's3'
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.bucket', 'local-opensearch-bucket'
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.base_path', 'raghraaj-local-1230'
+
+ // SSE-KMS configuration
+ if (findProperty("enableSseKms")) {
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.server_side_encryption_type', 'aws:kms'
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.server_side_encryption_kms_key_id', 'arn:aws:kms:us-east-1:389347062219:key/006ef490-5452-4f4d-8da3-a0cb7344ab59'
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.server_side_encryption_bucket_key_enabled', findProperty("sseBucketKeyEnabled") ?: 'true'
+ setting 'node.attr.remote_store.repository.my-s3-repo.settings.server_side_encryption_encryption_context', '{"identifier":"mustang"}'
+ }
+ }
if (findProperty("installedPlugins")) {
installedPlugins = Eval.me(installedPlugins)
diff --git a/modules/parquet-data-format/src/internalClusterTest/java/com/parquet/parquetdataformat/ParquetSegmentReplicationIT.java b/modules/parquet-data-format/src/internalClusterTest/java/com/parquet/parquetdataformat/ParquetSegmentReplicationIT.java
index a80ec3f158fa6..80090c0b178d2 100644
--- a/modules/parquet-data-format/src/internalClusterTest/java/com/parquet/parquetdataformat/ParquetSegmentReplicationIT.java
+++ b/modules/parquet-data-format/src/internalClusterTest/java/com/parquet/parquetdataformat/ParquetSegmentReplicationIT.java
@@ -314,80 +314,80 @@ public void testFormatAwareMetadataReplication() throws Exception {
/**
* Tests that replica can recover from remote store with Parquet files.
*/
-// public void testReplicaRecoveryWithParquetFiles() throws Exception {
-// internalCluster().startClusterManagerOnlyNode();
-// internalCluster().startDataOnlyNodes(2);
-// createReplicationIndex(INDEX_NAME, 1);
-//
-// // Index documents
-// for (int i = 0; i < 20; i++) {
-// client().prepareIndex(INDEX_NAME)
-// .setId(String.valueOf(i))
-// .setSource("id", String.valueOf(i), "field", "recovery" + i, "value", (long) i)
-// .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
-// .get();
-// }
-//
-// String primaryNode = getPrimaryNodeName(INDEX_NAME);
-// String replicaNode = getReplicaNodeName(INDEX_NAME);
-//
-// // Wait for initial replication
-// assertBusy(() -> {
-// IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
-// IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
-// assertEquals(
-// primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
-// replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
-// );
-// }, 30, TimeUnit.SECONDS);
-//
-// // Stop replica node to simulate failure
-// internalCluster().restartNode(replicaNode, new InternalTestCluster.RestartCallback() {
-// @Override
-// public Settings onNodeStopped(String nodeName) throws Exception {
-// // Index more documents on primary while replica is down
-// try {
-// for (int i = 20; i < 40; i++) {
-// client().prepareIndex(INDEX_NAME)
-// .setId(String.valueOf(i))
-// .setSource("id", String.valueOf(i), "field", "after_failure" + i, "value", (long) i)
-// .get();
-// }
-// client().admin().indices().prepareRefresh(INDEX_NAME).get();
-// } catch (Exception e) {
-// throw new RuntimeException(e);
-// }
-// return super.onNodeStopped(nodeName);
-// }
-// });
-//
-// ensureGreen(INDEX_NAME);
-//
-// // Verify replica recovered with Parquet files
-// assertBusy(() -> {
-// IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
-// IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
-//
-// // Verify checkpoints match after recovery
-// assertEquals(
-// "Replica should catch up after recovery",
-// primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
-// replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
-// );
-//
-// // Verify replica has Parquet files
-// RemoteSegmentStoreDirectory replicaRemoteDir = replicaShard.getRemoteDirectory();
-// Map replicaSegments =
-// replicaRemoteDir.getSegmentsUploadedToRemoteStore();
-//
-// Set formats = replicaSegments.keySet().stream()
-// .map(file -> new FileMetadata(file).dataFormat())
-// .collect(Collectors.toSet());
-//
-// assertTrue("Recovered replica should have Parquet files", formats.contains("parquet"));
-//
-// }, 60, TimeUnit.SECONDS);
-// }
+ public void testReplicaRecoveryWithParquetFiles() throws Exception {
+ internalCluster().startClusterManagerOnlyNode();
+ internalCluster().startDataOnlyNodes(2);
+ createReplicationIndex(INDEX_NAME, 1);
+
+ // Index documents
+ for (int i = 0; i < 20; i++) {
+ client().prepareIndex(INDEX_NAME)
+ .setId(String.valueOf(i))
+ .setSource("id", String.valueOf(i), "field", "recovery" + i, "value", (long) i)
+ .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
+ .get();
+ }
+
+ String primaryNode = getPrimaryNodeName(INDEX_NAME);
+ String replicaNode = getReplicaNodeName(INDEX_NAME);
+
+ // Wait for initial replication
+ assertBusy(() -> {
+ IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
+ IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
+ assertEquals(
+ primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
+ replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
+ );
+ }, 30, TimeUnit.SECONDS);
+
+ // Stop replica node to simulate failure
+ internalCluster().restartNode(replicaNode, new InternalTestCluster.RestartCallback() {
+ @Override
+ public Settings onNodeStopped(String nodeName) throws Exception {
+ // Index more documents on primary while replica is down
+ try {
+ for (int i = 20; i < 40; i++) {
+ client().prepareIndex(INDEX_NAME)
+ .setId(String.valueOf(i))
+ .setSource("id", String.valueOf(i), "field", "after_failure" + i, "value", (long) i)
+ .get();
+ }
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return super.onNodeStopped(nodeName);
+ }
+ });
+
+ ensureGreen(INDEX_NAME);
+
+ // Verify replica recovered with Parquet files
+ assertBusy(() -> {
+ IndexShard primaryShard = getIndexShard(primaryNode, INDEX_NAME);
+ IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME);
+
+ // Verify checkpoints match after recovery
+ assertEquals(
+ "Replica should catch up after recovery",
+ primaryShard.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
+ replicaShard.getLatestReplicationCheckpoint().getSegmentInfosVersion()
+ );
+
+ // Verify replica has Parquet files
+ RemoteSegmentStoreDirectory replicaRemoteDir = replicaShard.getRemoteDirectory();
+ Map replicaSegments =
+ replicaRemoteDir.getSegmentsUploadedToRemoteStore();
+
+ Set formats = replicaSegments.keySet().stream()
+ .map(file -> new FileMetadata(file).dataFormat())
+ .collect(Collectors.toSet());
+
+ assertTrue("Recovered replica should have Parquet files", formats.contains("parquet"));
+
+ }, 60, TimeUnit.SECONDS);
+ }
/**
* Tests that ReplicationCheckpoint contains format-aware metadata.
diff --git a/modules/parquet-data-format/src/test/java/com/parquet/parquetdataformat/vsr/VSRManagerTests.java b/modules/parquet-data-format/src/test/java/com/parquet/parquetdataformat/vsr/VSRManagerTests.java
index 046cc94f5433d..3c13cc511eb83 100644
--- a/modules/parquet-data-format/src/test/java/com/parquet/parquetdataformat/vsr/VSRManagerTests.java
+++ b/modules/parquet-data-format/src/test/java/com/parquet/parquetdataformat/vsr/VSRManagerTests.java
@@ -9,6 +9,7 @@
package com.parquet.parquetdataformat.vsr;
import com.parquet.parquetdataformat.bridge.ArrowExport;
+import com.parquet.parquetdataformat.bridge.ParquetFileMetadata;
import com.parquet.parquetdataformat.bridge.RustBridge;
import com.parquet.parquetdataformat.memory.ArrowBufferPool;
import com.parquet.parquetdataformat.writer.ParquetDocumentInput;
@@ -90,8 +91,8 @@ public void testVSRManagerInitializationAndActiveVSR() throws Exception {
// Flush before close (transitions VSR to FROZEN)
FlushIn flushIn = Mockito.mock(FlushIn.class);
- String flushResult = vsrManager.flush(flushIn);
- assertEquals("Flush should return filename", testFileName, flushResult);
+ ParquetFileMetadata flushResult = vsrManager.flush(flushIn);
+ assertNotNull("Flush should return metadata", flushResult);
assertEquals("VSR should be FROZEN after flush", VSRState.FROZEN, vsrManager.getActiveManagedVSR().getState());
// Now close should succeed
@@ -125,8 +126,8 @@ public void testDocumentAdditionThroughVSRManager() throws Exception {
// Follow proper VSRManager lifecycle: Write → Flush → Close
// Flush before close (transitions VSR to FROZEN)
FlushIn flushIn = Mockito.mock(FlushIn.class);
- String flushResult = vsrManager.flush(flushIn);
- assertEquals("Flush should return filename", testFileName, flushResult);
+ ParquetFileMetadata flushResult = vsrManager.flush(flushIn);
+ assertNotNull("Flush should return metadata", flushResult);
assertEquals("VSR should be FROZEN after flush", VSRState.FROZEN, vsrManager.getActiveManagedVSR().getState());
// Now close should succeed
@@ -142,9 +143,9 @@ public void testFlushThroughVSRManager() throws Exception {
// Flush through VSRManager (create mock FlushIn)
FlushIn flushIn = Mockito.mock(FlushIn.class);
- String result = vsrManager.flush(flushIn);
+ ParquetFileMetadata result = vsrManager.flush(flushIn);
- assertEquals("Flush should return filename", testFileName, result);
+ assertNotNull("Flush should return metadata", result);
// VSR should be FROZEN after flush
assertEquals("VSR should be FROZEN after flush",
@@ -166,9 +167,9 @@ public void testVSRManagerStateTransitionWorkflow() throws Exception {
// 3. Flush - should transition VSR to FROZEN
FlushIn flushIn = Mockito.mock(FlushIn.class);
- String flushResult = vsrManager.flush(flushIn);
+ ParquetFileMetadata flushResult = vsrManager.flush(flushIn);
- assertEquals("Flush should return filename", testFileName, flushResult);
+ assertNotNull("Flush should return metadata", flushResult);
assertEquals("VSR should be FROZEN after flush", VSRState.FROZEN, vsrManager.getActiveManagedVSR().getState());
assertTrue("VSR should be immutable when frozen", vsrManager.getActiveManagedVSR().isImmutable());
diff --git a/plugins/engine-datafusion/build.gradle b/plugins/engine-datafusion/build.gradle
index bb26ebd449612..edff9b4cba2ff 100644
--- a/plugins/engine-datafusion/build.gradle
+++ b/plugins/engine-datafusion/build.gradle
@@ -176,6 +176,12 @@ test {
systemProperty 'java.library.path', file('src/main/resources/native').absolutePath
}
+internalClusterTest {
+ // Add same JVM arguments for integration tests
+ jvmArgs += ["--add-opens", "java.base/java.nio=org.apache.arrow.memory.core,ALL-UNNAMED"]
+ systemProperty 'java.library.path', file('src/main/resources/native').absolutePath
+}
+
yamlRestTest {
systemProperty 'tests.security.manager', 'false'
// Disable yamlRestTest since this plugin doesn't have REST API endpoints
diff --git a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReader.java b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReader.java
index fb9a522adda07..ff5f1a3e7203e 100644
--- a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReader.java
+++ b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReader.java
@@ -24,6 +24,8 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
diff --git a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReaderManager.java b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReaderManager.java
index 9d60ee01ec006..471797fcdd6fa 100644
--- a/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReaderManager.java
+++ b/plugins/engine-datafusion/src/main/java/org/opensearch/datafusion/search/DatafusionReaderManager.java
@@ -12,7 +12,6 @@
import java.util.HashSet;
import java.util.Set;
import java.util.function.Consumer;
-import org.apache.lucene.search.ReferenceManager;
import org.opensearch.index.engine.CatalogSnapshotAwareRefreshListener;
import org.opensearch.index.engine.EngineReaderManager;
import org.opensearch.index.engine.FileDeletionListener;
diff --git a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionReaderManagerTests.java b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionReaderManagerTests.java
index 284cd5bc5e857..8f5c329e0935a 100644
--- a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionReaderManagerTests.java
+++ b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionReaderManagerTests.java
@@ -31,8 +31,10 @@
import org.opensearch.env.Environment;
import org.opensearch.index.engine.exec.*;
import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
+import org.opensearch.index.engine.exec.coord.CompositeEngineCatalogSnapshot;
import org.opensearch.index.engine.exec.coord.CompositeEngine;
import org.opensearch.index.engine.exec.coord.IndexFileDeleter;
+import org.opensearch.index.engine.exec.coord.Segment;
import org.opensearch.index.shard.ShardPath;
import org.opensearch.search.aggregations.SearchResultsCollector;
import org.opensearch.test.OpenSearchTestCase;
@@ -103,14 +105,14 @@ public void testInitialReaderCreation() throws IOException {
DatafusionReaderManager readerManager = engine.getReferenceManager(INTERNAL);
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(1);
+ Segment segment = new Segment(1);
WriterFileSet writerFileSet = new WriterFileSet(parquetDir, 1, 4);
writerFileSet.add(parquetDir + "/parquet_file_generation_0.parquet");
writerFileSet.add(parquetDir + "/parquet_file_generation_1.parquet");
segment.addSearchableFiles(getMockDataFormat().name(), writerFileSet);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher = engine.acquireSearcher("test");
DatafusionReader reader = searcher.getReader();
@@ -134,13 +136,13 @@ public void testMultipleSearchersShareSameReader() throws IOException {
DatafusionReaderManager readerManager = engine.getReferenceManager(INTERNAL);
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(1);
+ Segment segment = new Segment(1);
WriterFileSet writerFileSet = new WriterFileSet(parquetDir, 1, 2);
writerFileSet.add(parquetDir + "/parquet_file_generation_0.parquet");
segment.addSearchableFiles(getMockDataFormat().name(), writerFileSet);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher1 = engine.acquireSearcher("test1");
DatafusionSearcher searcher2 = engine.acquireSearcher("test2");
@@ -165,13 +167,13 @@ public void testReaderSurvivesPartialSearcherClose() throws IOException {
DatafusionReaderManager readerManager = engine.getReferenceManager(INTERNAL);
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(1);
+ Segment segment = new Segment(1);
WriterFileSet writerFileSet = new WriterFileSet(parquetDir, 1, 2);
writerFileSet.add(parquetDir + "/parquet_file_generation_0.parquet");
segment.addSearchableFiles(getMockDataFormat().name(), writerFileSet);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher1 = engine.acquireSearcher("test1");
DatafusionSearcher searcher2 = engine.acquireSearcher("test2");
@@ -197,14 +199,14 @@ public void testRefreshCreatesNewReader() throws IOException {
Path parquetDir = shardPath.getDataPath().resolve("parquet");
// Initial refresh
- CatalogSnapshot.Segment segment1 = new CatalogSnapshot.Segment(1);
+ Segment segment1 = new Segment(1);
WriterFileSet writerFileSet1 = new WriterFileSet(parquetDir, 1, 2);
addFilesToShardPath(shardPath, "parquet_file_generation_0.parquet");
writerFileSet1.add(parquetDir + "/parquet_file_generation_0.parquet");
segment1.addSearchableFiles(getMockDataFormat().name(), writerFileSet1);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment1), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment1), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher1 = engine.acquireSearcher("test1");
DatafusionReader reader1 = searcher1.getReader();
@@ -212,14 +214,14 @@ public void testRefreshCreatesNewReader() throws IOException {
// Add new file and refresh
addFilesToShardPath(shardPath, "parquet_file_generation_1.parquet");
- CatalogSnapshot.Segment segment2 = new CatalogSnapshot.Segment(2);
+ Segment segment2 = new Segment(2);
WriterFileSet writerFileSet2 = new WriterFileSet(parquetDir, 2, 4);
writerFileSet2.add(parquetDir + "/parquet_file_generation_0.parquet");
writerFileSet2.add(parquetDir + "/parquet_file_generation_1.parquet");
segment2.addSearchableFiles(getMockDataFormat().name(), writerFileSet2);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(2, 2, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(2, 2, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher2 = engine.acquireSearcher("test2");
DatafusionReader reader2 = searcher2.getReader();
@@ -246,13 +248,13 @@ public void testDecRefAfterCloseThrowsException() throws IOException {
DatafusionReaderManager readerManager = engine.getReferenceManager(INTERNAL);
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(1);
+ Segment segment = new Segment(1);
WriterFileSet writerFileSet = new WriterFileSet(parquetDir, 1, 4);
writerFileSet.add(parquetDir + "/parquet_file_generation_2.parquet");
segment.addSearchableFiles(getMockDataFormat().name(), writerFileSet);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher = engine.acquireSearcher("test");
DatafusionReader reader = searcher.getReader();
@@ -276,14 +278,14 @@ public void testReaderClosesAfterSearchRelease() throws IOException {
DatafusionReaderManager readerManager = engine.getReferenceManager(INTERNAL);
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(1);
+ Segment segment = new Segment(1);
WriterFileSet writerFileSet = new WriterFileSet(parquetDir, 1, 6);
writerFileSet.add(parquetDir + "/parquet_file_generation_2.parquet");
writerFileSet.add(parquetDir + "/parquet_file_generation_1.parquet");
segment.addSearchableFiles(getMockDataFormat().name(), writerFileSet);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment), new HashMap<>(), noOpFileDeleterSupplier)));
// DatafusionReader readerR1 = readerManager.acquire();
DatafusionSearcher datafusionSearcherS1 = engine.acquireSearcher("Search");
@@ -299,14 +301,14 @@ public void testReaderClosesAfterSearchRelease() throws IOException {
addFilesToShardPath(shardPath, "parquet_file_generation_0.parquet");
// now trigger refresh to have new Reader with F2, F3
- CatalogSnapshot.Segment segment2 = new CatalogSnapshot.Segment(2);
+ Segment segment2 = new Segment(2);
WriterFileSet writerFileSet2 = new WriterFileSet(parquetDir, 2, 4);
writerFileSet2.add(parquetDir + "/parquet_file_generation_1.parquet");
writerFileSet2.add(parquetDir + "/parquet_file_generation_0.parquet");
segment2.addSearchableFiles(getMockDataFormat().name(), writerFileSet2);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(2, 2, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(2, 2, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
// now check if new Reader is created with F2, F3
// DatafusionReader readerR2 = readerManager.acquire();
@@ -345,13 +347,13 @@ public void testSearch() throws Exception {
// Initial refresh - files are in the parquet subdirectory
Path parquetDir = shardPath.getDataPath().resolve("parquet");
- CatalogSnapshot.Segment segment1 = new CatalogSnapshot.Segment(0);
+ Segment segment1 = new Segment(0);
WriterFileSet writerFileSet1 = new WriterFileSet(parquetDir, 0, 2);
writerFileSet1.add(parquetDir + "/parquet_file_generation_0.parquet");
segment1.addSearchableFiles(getMockDataFormat().name(), writerFileSet1);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(1, 1, List.of(segment1), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(1, 1, List.of(segment1), new HashMap<>(), noOpFileDeleterSupplier)));
DatafusionSearcher searcher1 = engine.acquireSearcher("search");
DatafusionReader reader1 = searcher1.getReader();
@@ -375,13 +377,13 @@ public void testSearch() throws Exception {
logger.info("AFTER REFRESH");
addFilesToShardPath(shardPath, "parquet_file_generation_1.parquet");
- CatalogSnapshot.Segment segment2 = new CatalogSnapshot.Segment(1);
+ Segment segment2 = new Segment(1);
WriterFileSet writerFileSet2 = new WriterFileSet(parquetDir, 1, 2);
writerFileSet2.add(parquetDir + "/parquet_file_generation_1.parquet");
segment2.addSearchableFiles(getMockDataFormat().name(), writerFileSet2);
readerManager.afterRefresh(true,
- () -> getCatalogSnapshotRef(new CatalogSnapshot(2, 1, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
+ () -> getCatalogSnapshotRef(new CompositeEngineCatalogSnapshot(2, 1, List.of(segment2), new HashMap<>(), noOpFileDeleterSupplier)));
expectedResults = new HashMap<>();
expectedResults.put("min", 3L);
diff --git a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionRemoteStoreRecoveryTests.java b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionRemoteStoreRecoveryTests.java
new file mode 100644
index 0000000000000..ee7c5ac15cd23
--- /dev/null
+++ b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionRemoteStoreRecoveryTests.java
@@ -0,0 +1,583 @@
+/*
+ * 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.datafusion;
+
+import com.parquet.parquetdataformat.ParquetDataFormatPlugin;
+import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest;
+import org.opensearch.action.support.PlainActionFuture;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.index.store.RemoteSegmentStoreDirectory;
+import org.opensearch.index.store.UploadedSegmentMetadata;
+import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata;
+import org.opensearch.indices.replication.common.ReplicationType;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.test.junit.annotations.TestLogging;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.opensearch.index.store.CompositeStoreDirectory;
+
+import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
+
+/**
+ * Integration tests for DataFusion engine remote store recovery scenarios.
+ * Tests format-aware metadata preservation, CatalogSnapshot recovery, and
+ * remote store recovery validation with Parquet/Arrow files.
+ */
+@TestLogging(
+ value = "org.opensearch.index.shard:DEBUG,org.opensearch.index.store:DEBUG,org.opensearch.datafusion:DEBUG",
+ reason = "Validate DataFusion recovery with format-aware metadata"
+)
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
+public class DataFusionRemoteStoreRecoveryTests extends OpenSearchIntegTestCase {
+
+ protected static final String REPOSITORY_NAME = "test-remote-store-repo";
+ protected static final String INDEX_NAME = "datafusion-test-index";
+
+ protected Path repositoryPath;
+
+ @Override
+ protected Collection> nodePlugins() {
+ return List.of(DataFusionPlugin.class, ParquetDataFormatPlugin.class);
+ }
+
+ @Before
+ public void setup() {
+ repositoryPath = randomRepoPath().toAbsolutePath();
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put(remoteStoreClusterSettings(REPOSITORY_NAME, repositoryPath))
+ .put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true)
+ .build();
+ }
+
+ @Override
+ public Settings indexSettings() {
+ return Settings.builder()
+ .put(super.indexSettings())
+ .put("index.queries.cache.enabled", false)
+ .put("index.refresh_interval", -1)
+ .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
+ .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+ .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
+ .put("index.optimized.enabled", true)
+ .build();
+ }
+
+ @Override
+ protected void beforeIndexDeletion() throws Exception {
+ logger.info("--> Skipping beforeIndexDeletion cleanup to avoid DataFusion engine type conflicts");
+ }
+
+ @Override
+ protected void ensureClusterSizeConsistency() {}
+
+ @Override
+ protected void ensureClusterStateConsistency() {}
+
+ private IndexShard getIndexShard(String nodeName, String indexName) {
+ return internalCluster().getInstance(org.opensearch.indices.IndicesService.class, nodeName)
+ .indexServiceSafe(internalCluster().clusterService(nodeName).state().metadata().index(indexName).getIndex())
+ .getShard(0);
+ }
+
+ private void validateRemoteStoreSegments(IndexShard shard, String stageName) {
+ RemoteSegmentStoreDirectory remoteDir = shard.getRemoteDirectory();
+ assertNotNull("RemoteSegmentStoreDirectory should not be null", remoteDir);
+
+ Map uploadedSegmentsRaw = remoteDir.getSegmentsUploadedToRemoteStore();
+ if (uploadedSegmentsRaw.isEmpty()) {
+ logger.warn("--> No segments uploaded yet at stage: {}", stageName);
+ return;
+ }
+
+ Map uploadedSegments = uploadedSegmentsRaw.entrySet().stream()
+ .collect(Collectors.toMap(e -> new FileMetadata(e.getKey()), Map.Entry::getValue));
+
+ for (FileMetadata fileMetadata : uploadedSegments.keySet()) {
+ assertNotNull("FileMetadata should have format information", fileMetadata.dataFormat());
+ assertFalse("Format should not be empty", fileMetadata.dataFormat().isEmpty());
+ }
+ }
+
+ private long validateLocalShardFiles(IndexShard shard, String stageName) {
+ try {
+ CompositeStoreDirectory compositeDir = shard.store().compositeStoreDirectory();
+ if (compositeDir != null) {
+ FileMetadata[] allFiles = compositeDir.listFileMetadata();
+ return Arrays.stream(allFiles).filter(fm -> "parquet".equals(fm.dataFormat())).count();
+ } else {
+ String[] files = shard.store().directory().listAll();
+ return Arrays.stream(files).filter(f -> f.contains("parquet") || f.endsWith(".parquet")).count();
+ }
+ } catch (IOException e) {
+ logger.warn("--> Failed to list local shard files at stage {}: {}", stageName, e.getMessage());
+ return -1;
+ }
+ }
+
+ private void validateCatalogSnapshot(IndexShard shard, String stageName) {
+ RemoteSegmentStoreDirectory remoteDir = shard.getRemoteDirectory();
+ assertNotNull("RemoteSegmentStoreDirectory should not be null", remoteDir);
+
+ try {
+ RemoteSegmentMetadata metadata = remoteDir.readLatestMetadataFile();
+ if (metadata == null) {
+ logger.warn("--> RemoteSegmentMetadata not found at stage {}", stageName);
+ return;
+ }
+
+ byte[] catalogSnapshotBytes = metadata.getSegmentInfosBytes();
+ if (catalogSnapshotBytes != null) {
+ assertTrue("CatalogSnapshot bytes should not be empty", catalogSnapshotBytes.length > 0);
+ }
+
+ var checkpoint = metadata.getReplicationCheckpoint();
+ if (checkpoint != null) {
+ assertTrue("Checkpoint version should be positive", checkpoint.getSegmentInfosVersion() > 0);
+ }
+ } catch (IOException e) {
+ logger.warn("--> Failed to read metadata at stage {}: {}", stageName, e.getMessage());
+ }
+ }
+
+ /**
+ * Tests DataFusion engine recovery from remote store with format-aware metadata preservation.
+ */
+ public void testDataFusionWithRemoteStoreRecovery() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(1);
+ ensureStableCluster(2);
+
+ String mappings = "{ \"properties\": { \"message\": { \"type\": \"long\" }, \"message2\": { \"type\": \"long\" }, \"message3\": { \"type\": \"long\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME).setSettings(indexSettings()).setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ client().prepareIndex(INDEX_NAME).setId("1").setSource("{ \"message\": 4, \"message2\": 3, \"message3\": 4 }", MediaTypeRegistry.JSON).get();
+ client().prepareIndex(INDEX_NAME).setId("2").setSource("{ \"message\": 3, \"message2\": 4, \"message3\": 5 }", MediaTypeRegistry.JSON).get();
+ client().prepareIndex(INDEX_NAME).setId("3").setSource("{ \"message\": 5, \"message2\": 2, \"message3\": 3 }", MediaTypeRegistry.JSON).get();
+
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+
+ String dataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard indexShard = getIndexShard(dataNodeName, INDEX_NAME);
+ validateRemoteStoreSegments(indexShard, "before recovery");
+ validateCatalogSnapshot(indexShard, "before recovery");
+
+ // Capture state before recovery for comparison
+ long docCountBeforeRecovery = indexShard.docStats().getCount();
+ long localFilesBeforeRecovery = validateLocalShardFiles(indexShard, "before recovery");
+
+ String clusterUUID = clusterService().state().metadata().clusterUUID();
+ internalCluster().stopRandomDataNode();
+ ensureRed(INDEX_NAME);
+
+ internalCluster().startDataOnlyNode();
+ ensureStableCluster(2);
+
+ assertAcked(client().admin().indices().prepareClose(INDEX_NAME));
+ client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(INDEX_NAME).restoreAllShards(true), PlainActionFuture.newFuture());
+ ensureGreen(INDEX_NAME);
+ client().admin().indices().prepareFlush(INDEX_NAME).setForce(true).get();
+
+ assertEquals("Cluster UUID should remain same", clusterUUID, clusterService().state().metadata().clusterUUID());
+
+ String newDataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard recoveredIndexShard = getIndexShard(newDataNodeName, INDEX_NAME);
+ validateRemoteStoreSegments(recoveredIndexShard, "after recovery");
+ validateCatalogSnapshot(recoveredIndexShard, "after recovery");
+
+ long localFilesAfterRecovery = validateLocalShardFiles(recoveredIndexShard, "after recovery");
+ assertTrue("Should have local files after recovery", localFilesAfterRecovery >= 0);
+
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ long docCountAfterRecovery = recoveredIndexShard.docStats().getCount();
+
+ // Verify before/after comparison
+ assertEquals("Doc count should be same before and after recovery", docCountBeforeRecovery, docCountAfterRecovery);
+ assertEquals("Local file count should be same before and after recovery", localFilesBeforeRecovery, localFilesAfterRecovery);
+
+ assertAcked(client().admin().indices().prepareDelete(INDEX_NAME).get());
+ }
+
+ /**
+ * Tests DataFusion recovery with multiple Parquet generation files.
+ */
+ public void testDataFusionRecoveryWithMultipleParquetGenerations() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(1);
+ ensureStableCluster(2);
+
+ String mappings = "{ \"properties\": { \"message\": { \"type\": \"long\" }, \"message2\": { \"type\": \"long\" }, \"generation\": { \"type\": \"keyword\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME).setSettings(indexSettings()).setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ String dataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard indexShard = getIndexShard(dataNodeName, INDEX_NAME);
+
+ int numGenerations = 4;
+ for (int gen = 1; gen <= numGenerations; gen++) {
+ for (int i = 1; i <= 3; i++) {
+ client().prepareIndex(INDEX_NAME).setId("gen" + gen + "_doc" + i)
+ .setSource("{ \"message\": " + (gen * 100 + i) + ", \"message2\": " + (gen * 200 + i) + ", \"generation\": \"gen" + gen + "\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ Thread.sleep(500);
+ }
+
+ validateRemoteStoreSegments(indexShard, "before recovery");
+ RemoteSegmentStoreDirectory remoteDir = indexShard.getRemoteDirectory();
+ Map uploadedSegments = remoteDir.getSegmentsUploadedToRemoteStore().entrySet().stream()
+ .collect(Collectors.toMap(e -> new FileMetadata(e.getKey()), Map.Entry::getValue));
+ long parquetFileCount = uploadedSegments.keySet().stream().filter(fm -> "parquet".equals(fm.dataFormat())).count();
+ assertTrue("Should have multiple Parquet generation files", parquetFileCount >= numGenerations);
+
+ // Capture state before recovery for comparison
+ long docCountBeforeRecovery = indexShard.docStats().getCount();
+ long localFilesBeforeRecovery = validateLocalShardFiles(indexShard, "before recovery");
+
+ String clusterUUID = clusterService().state().metadata().clusterUUID();
+ internalCluster().stopRandomDataNode();
+ ensureRed(INDEX_NAME);
+
+ internalCluster().startDataOnlyNode();
+ ensureStableCluster(2);
+
+ assertAcked(client().admin().indices().prepareClose(INDEX_NAME));
+ client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(INDEX_NAME).restoreAllShards(true), PlainActionFuture.newFuture());
+ ensureGreen(INDEX_NAME);
+
+ String newDataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard recoveredIndexShard = getIndexShard(newDataNodeName, INDEX_NAME);
+ validateRemoteStoreSegments(recoveredIndexShard, "after recovery");
+
+ RemoteSegmentStoreDirectory recoveredRemoteDir = recoveredIndexShard.getRemoteDirectory();
+ Map recoveredSegments = recoveredRemoteDir.getSegmentsUploadedToRemoteStore().entrySet().stream()
+ .collect(Collectors.toMap(e -> new FileMetadata(e.getKey()), Map.Entry::getValue));
+ long recoveredParquetFileCount = recoveredSegments.keySet().stream().filter(fm -> "parquet".equals(fm.dataFormat())).count();
+ assertEquals("Should recover same number of Parquet files", parquetFileCount, recoveredParquetFileCount);
+
+ long localFilesAfterRecovery = validateLocalShardFiles(recoveredIndexShard, "after recovery");
+ assertTrue("Should have local files after recovery", localFilesAfterRecovery >= 0);
+
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ long docCountAfterRecovery = recoveredIndexShard.docStats().getCount();
+
+ // Verify before/after comparison
+ assertEquals("Doc count should be same before and after recovery", docCountBeforeRecovery, docCountAfterRecovery);
+ assertEquals("Local file count should be same before and after recovery", localFilesBeforeRecovery, localFilesAfterRecovery);
+ assertEquals("Cluster UUID should remain same", clusterUUID, clusterService().state().metadata().clusterUUID());
+ }
+
+ /**
+ * Tests DataFusion replica promotion to primary with Parquet format preservation.
+ */
+ public void testDataFusionReplicaPromotionToPrimary() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(2);
+ ensureStableCluster(3);
+
+ String mappings = "{ \"properties\": { \"message\": { \"type\": \"long\" }, \"phase\": { \"type\": \"keyword\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME)
+ .setSettings(Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build())
+ .setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ for (int i = 1; i <= 5; i++) {
+ client().prepareIndex(INDEX_NAME).setId("primary_doc" + i)
+ .setSource("{ \"message\": " + (i * 100) + ", \"phase\": \"primary\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ ensureGreen(INDEX_NAME);
+
+ var clusterState = clusterService().state();
+ var shardRouting = clusterState.routingTable().index(INDEX_NAME).shard(0);
+ String primaryNodeId = shardRouting.primaryShard().currentNodeId();
+ String replicaNodeId = shardRouting.replicaShards().get(0).currentNodeId();
+
+ String primaryNodeName = null, replicaNodeName = null;
+ for (String nodeName : internalCluster().getNodeNames()) {
+ String nodeId = internalCluster().clusterService(nodeName).localNode().getId();
+ if (nodeId.equals(primaryNodeId)) primaryNodeName = nodeName;
+ else if (nodeId.equals(replicaNodeId)) replicaNodeName = nodeName;
+ }
+
+ IndexShard replicaShard = internalCluster().getInstance(org.opensearch.indices.IndicesService.class, replicaNodeName)
+ .indexServiceSafe(resolveIndex(INDEX_NAME)).getShard(0);
+ Thread.sleep(2000);
+ validateRemoteStoreSegments(replicaShard, "replica before promotion");
+
+ // Capture state before promotion for comparison
+ long docCountBeforePromotion = replicaShard.docStats().getCount();
+ long localFilesBeforePromotion = validateLocalShardFiles(replicaShard, "replica before promotion");
+
+ internalCluster().stopRandomNode(org.opensearch.test.InternalTestCluster.nameFilter(primaryNodeName));
+ ensureStableCluster(2);
+ ensureYellow(INDEX_NAME);
+
+ IndexShard promotedShard = internalCluster().getInstance(org.opensearch.indices.IndicesService.class, replicaNodeName)
+ .indexServiceSafe(resolveIndex(INDEX_NAME)).getShard(0);
+ assertTrue("Former replica should now be primary", promotedShard.routingEntry().primary());
+ validateRemoteStoreSegments(promotedShard, "after promotion");
+
+ Set formats = promotedShard.getRemoteDirectory().getSegmentsUploadedToRemoteStore().entrySet().stream()
+ .map(e -> new FileMetadata(e.getKey()).dataFormat()).collect(Collectors.toSet());
+ assertTrue("Promoted primary should have Parquet files", formats.contains("parquet"));
+
+ for (int i = 1; i <= 3; i++) {
+ client().prepareIndex(INDEX_NAME).setId("promoted_doc" + i)
+ .setSource("{ \"message\": " + (i * 200) + ", \"phase\": \"promoted\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+
+ long localFilesAfterPromotion = validateLocalShardFiles(promotedShard, "after promotion and new docs");
+ assertTrue("Should have local files after promotion", localFilesAfterPromotion >= 0);
+
+ // Verify final state (5 original + 3 new docs)
+ assertEquals("Final document count should match", 8, promotedShard.docStats().getCount());
+ // Local files should increase after adding new docs
+ assertTrue("Local files should exist after new writes", localFilesAfterPromotion >= localFilesBeforePromotion);
+ }
+
+ /**
+ * Tests cluster recovery from remote translog when no flush/refresh is performed.
+ */
+ public void testClusterRecoveryFromTranslogWithoutFlush() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(1);
+ ensureStableCluster(2);
+
+ String mappings = "{ \"properties\": { \"value\": { \"type\": \"long\" }, \"name\": { \"type\": \"keyword\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME)
+ .setSettings(Settings.builder().put(indexSettings()).put("index.translog.durability", "request").build())
+ .setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ int numDocs = 10;
+ for (int i = 1; i <= numDocs; i++) {
+ client().prepareIndex(INDEX_NAME).setId("doc" + i)
+ .setSource("{ \"value\": " + (i * 100) + ", \"name\": \"doc" + i + "\" }", MediaTypeRegistry.JSON).get();
+ }
+ // Intentionally NOT calling flush or refresh - documents exist only in translog
+ Thread.sleep(1000);
+
+ String dataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard indexShard = getIndexShard(dataNodeName, INDEX_NAME);
+ assertTrue("Translog should have uncommitted operations", indexShard.translogStats().getUncommittedOperations() >= numDocs);
+
+ String clusterUUID = clusterService().state().metadata().clusterUUID();
+ internalCluster().stopRandomDataNode();
+ ensureRed(INDEX_NAME);
+
+ internalCluster().startDataOnlyNode();
+ ensureStableCluster(2);
+
+ assertAcked(client().admin().indices().prepareClose(INDEX_NAME));
+ client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(INDEX_NAME).restoreAllShards(true), PlainActionFuture.newFuture());
+ ensureGreen(INDEX_NAME);
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+
+ String newDataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard recoveredShard = getIndexShard(newDataNodeName, INDEX_NAME);
+
+ assertBusy(() -> assertTrue("Translog should have processed operations",
+ recoveredShard.translogStats().estimatedNumberOfOperations() >= 0), 30, TimeUnit.SECONDS);
+
+ long parquetFilesAfterRecovery = validateLocalShardFiles(recoveredShard, "after recovery");
+ assertTrue("Should have local files after recovery", parquetFilesAfterRecovery >= 0);
+ assertEquals("Document count should match", numDocs, recoveredShard.docStats().getCount());
+ assertEquals("Cluster UUID should remain same", clusterUUID, clusterService().state().metadata().clusterUUID());
+
+ assertAcked(client().admin().indices().prepareDelete(INDEX_NAME).get());
+ }
+
+ /**
+ * Tests replica promotion to primary with translog replay for uncommitted operations.
+ */
+ public void testReplicaPromotionWithTranslogReplay() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(2);
+ ensureStableCluster(3);
+
+ String mappings = "{ \"properties\": { \"value\": { \"type\": \"long\" }, \"phase\": { \"type\": \"keyword\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME)
+ .setSettings(Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
+ .put("index.translog.durability", "request").build())
+ .setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ int initialDocs = 5;
+ for (int i = 1; i <= initialDocs; i++) {
+ client().prepareIndex(INDEX_NAME).setId("initial_doc" + i)
+ .setSource("{ \"value\": " + (i * 100) + ", \"phase\": \"initial\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ ensureGreen(INDEX_NAME);
+
+ int uncommittedDocs = 7;
+ for (int i = 1; i <= uncommittedDocs; i++) {
+ client().prepareIndex(INDEX_NAME).setId("uncommitted_doc" + i)
+ .setSource("{ \"value\": " + (i * 200) + ", \"phase\": \"uncommitted\" }", MediaTypeRegistry.JSON).get();
+ }
+ // Intentionally NOT calling flush or refresh - docs exist only in translog
+ Thread.sleep(1000);
+
+ var clusterState = clusterService().state();
+ var shardRouting = clusterState.routingTable().index(INDEX_NAME).shard(0);
+ String primaryNodeId = shardRouting.primaryShard().currentNodeId();
+ String replicaNodeId = shardRouting.replicaShards().get(0).currentNodeId();
+
+ String primaryNodeName = null, replicaNodeName = null;
+ for (String nodeName : internalCluster().getNodeNames()) {
+ String nodeId = internalCluster().clusterService(nodeName).localNode().getId();
+ if (nodeId.equals(primaryNodeId)) primaryNodeName = nodeName;
+ else if (nodeId.equals(replicaNodeId)) replicaNodeName = nodeName;
+ }
+ assertNotNull("Primary node name should be found", primaryNodeName);
+ assertNotNull("Replica node name should be found", replicaNodeName);
+
+ IndexShard primaryShard = internalCluster().getInstance(org.opensearch.indices.IndicesService.class, primaryNodeName)
+ .indexServiceSafe(resolveIndex(INDEX_NAME)).getShard(0);
+ assertTrue("Primary should have uncommitted translog operations", primaryShard.translogStats().getUncommittedOperations() >= uncommittedDocs);
+
+ IndexShard replicaShard = internalCluster().getInstance(org.opensearch.indices.IndicesService.class, replicaNodeName)
+ .indexServiceSafe(resolveIndex(INDEX_NAME)).getShard(0);
+ long replicaFilesBeforePromotion = validateLocalShardFiles(replicaShard, "replica before promotion");
+
+ String finalReplicaNodeName = replicaNodeName;
+ internalCluster().stopRandomNode(org.opensearch.test.InternalTestCluster.nameFilter(primaryNodeName));
+ ensureStableCluster(2);
+
+ assertBusy(() -> {
+ var health = client().admin().cluster().prepareHealth(INDEX_NAME).get();
+ assertTrue("Index should not be red", health.getStatus() != org.opensearch.cluster.health.ClusterHealthStatus.RED);
+ }, 30, TimeUnit.SECONDS);
+ ensureYellow(INDEX_NAME);
+
+ IndexShard promotedShard = internalCluster().getInstance(org.opensearch.indices.IndicesService.class, finalReplicaNodeName)
+ .indexServiceSafe(resolveIndex(INDEX_NAME)).getShard(0);
+ assertTrue("Former replica should now be primary", promotedShard.routingEntry().primary());
+
+ assertBusy(() -> assertTrue("Translog should have processed operations",
+ promotedShard.translogStats().estimatedNumberOfOperations() >= 0), 30, TimeUnit.SECONDS);
+
+ validateRemoteStoreSegments(promotedShard, "after promotion");
+ long promotedFilesAfterPromotion = validateLocalShardFiles(promotedShard, "after promotion");
+ assertTrue("Promoted primary should have local files", promotedFilesAfterPromotion >= 0);
+
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ assertEquals("Document count should include all documents", initialDocs + uncommittedDocs, promotedShard.docStats().getCount());
+
+ int newDocs = 3;
+ for (int i = 1; i <= newDocs; i++) {
+ client().prepareIndex(INDEX_NAME).setId("post_promotion_doc" + i)
+ .setSource("{ \"value\": " + (i * 300) + ", \"phase\": \"post_promotion\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+
+ assertAcked(client().admin().indices().prepareDelete(INDEX_NAME).get());
+ }
+
+ /**
+ * Tests DataFusion primary restart with extra local commits.
+ */
+ public void testDataFusionPrimaryRestartWithExtraCommits() throws Exception {
+ internalCluster().startClusterManagerOnlyNodes(1);
+ internalCluster().startDataOnlyNodes(1);
+ ensureStableCluster(2);
+
+ String mappings = "{ \"properties\": { \"message\": { \"type\": \"long\" }, \"stage\": { \"type\": \"keyword\" } } }";
+ assertAcked(client().admin().indices().prepareCreate(INDEX_NAME).setSettings(indexSettings()).setMapping(mappings).get());
+ ensureGreen(INDEX_NAME);
+
+ for (int i = 1; i <= 4; i++) {
+ client().prepareIndex(INDEX_NAME).setId("initial_doc" + i)
+ .setSource("{ \"message\": " + (i * 100) + ", \"stage\": \"initial\" }", MediaTypeRegistry.JSON).get();
+ }
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+
+ String dataNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard indexShard = getIndexShard(dataNodeName, INDEX_NAME);
+ validateRemoteStoreSegments(indexShard, "initial upload");
+
+ // Capture state before extra docs and restart for comparison
+ long docCountAfterInitial = indexShard.docStats().getCount();
+ long localFilesAfterInitial = validateLocalShardFiles(indexShard, "after initial flush");
+
+ for (int i = 1; i <= 3; i++) {
+ client().prepareIndex(INDEX_NAME).setId("extra_doc" + i)
+ .setSource("{ \"message\": " + (i * 300) + ", \"stage\": \"extra\" }", MediaTypeRegistry.JSON).get();
+ }
+
+ try {
+ org.apache.lucene.index.SegmentInfos latestCommit = org.apache.lucene.index.SegmentInfos.readLatestCommit(indexShard.store().directory());
+ latestCommit.commit(indexShard.store().directory());
+ latestCommit.commit(indexShard.store().directory());
+ } catch (Exception e) {
+ logger.warn("--> Could not create extra commits: {}", e.getMessage());
+ }
+
+ String nodeToRestart = internalCluster().getDataNodeNames().iterator().next();
+ internalCluster().restartNode(nodeToRestart, new org.opensearch.test.InternalTestCluster.RestartCallback() {
+ @Override
+ public Settings onNodeStopped(String nodeName) throws Exception {
+ return super.onNodeStopped(nodeName);
+ }
+ });
+ ensureStableCluster(2);
+ ensureGreen(INDEX_NAME);
+
+ String restartedNodeName = internalCluster().getDataNodeNames().iterator().next();
+ IndexShard recoveredShard = getIndexShard(restartedNodeName, INDEX_NAME);
+ validateRemoteStoreSegments(recoveredShard, "after restart");
+
+ long localFilesAfterRecovery = validateLocalShardFiles(recoveredShard, "after restart");
+ assertTrue("Should have local files after restart", localFilesAfterRecovery >= 0);
+
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+ long docCountAfterRestart = recoveredShard.docStats().getCount();
+
+ // Verify doc count: initial 4 + extra 3 = 7
+ assertEquals("Document count should match total docs after restart", 7, docCountAfterRestart);
+ // Local files should be at least as many as after initial flush
+ assertTrue("Local files should be preserved after restart", localFilesAfterRecovery >= localFilesAfterInitial);
+
+ client().prepareIndex(INDEX_NAME).setId("post_recovery_doc")
+ .setSource("{ \"message\": 999, \"stage\": \"post_recovery\" }", MediaTypeRegistry.JSON).get();
+ client().admin().indices().prepareFlush(INDEX_NAME).get();
+ client().admin().indices().prepareRefresh(INDEX_NAME).get();
+
+ assertEquals("Final document count should match", 8, recoveredShard.docStats().getCount());
+ }
+}
diff --git a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionServiceTests.java b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionServiceTests.java
index f6b5c176e41bb..08bb2b2bebc30 100644
--- a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionServiceTests.java
+++ b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionServiceTests.java
@@ -150,7 +150,7 @@ public void testQueryPhaseExecutor() throws IOException {
Index index = new Index("index-7", "index-7");
final Path path = Path.of(resourceUrl.toURI()).resolve("index-7").resolve("0");
ShardPath shardPath = new ShardPath(false, path, path, new ShardId(index, 0));
- DatafusionEngine engine = new DatafusionEngine(DataFormat.CSV, List.of(new FileMetadata(DataFormat.CSV.toString(), "generation-1.parquet")), service, shardPath);
+ DatafusionEngine engine = new DatafusionEngine(DataFormat.CSV, List.of(new FileMetadata(DataFormat.CSV.getName(), "generation-1.parquet")), service, shardPath);
datafusionSearcher = engine.acquireSearcher("search");
byte[] protoContent;
@@ -289,7 +289,6 @@ public void testQueryThenFetchE2ETest() throws IOException, URISyntaxException,
final Path path = Path.of(resourceUrl.toURI()).resolve("index-7").resolve("0");
ShardPath shardPath = new ShardPath(false, path, path, new ShardId(index, 0));
DatafusionEngine engine = new DatafusionEngine(DataFormat.CSV, List.of(new FileMetadata(DataFormat.CSV.toString(), "generation-1.parquet"), new FileMetadata(DataFormat.CSV.toString(), "generation-2.parquet")), service, shardPath);
-
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true).source(new SearchSourceBuilder().size(9).fetchSource(List.of("message").toArray(String[]::new), null));
ShardSearchRequest shardSearchRequest = new ShardSearchRequest(
OriginalIndices.NONE,
diff --git a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionSingleNodeTests.java b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionSingleNodeTests.java
index 505a55e1514ec..98c0939122b84 100644
--- a/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionSingleNodeTests.java
+++ b/plugins/engine-datafusion/src/test/java/org/opensearch/datafusion/DataFusionSingleNodeTests.java
@@ -31,12 +31,14 @@
import java.util.List;
import java.util.Locale;
+
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST)
public class DataFusionSingleNodeTests extends OpenSearchSingleNodeTestCase {
private static final String INDEX_MAPPING_JSON = "clickbench_index_mapping.json";
private static final String DATA = "clickbench.json";
private final String indexName = "hits";
+ private static final String REPOSITORY_NAME = "test-remote-store-repo";
@Override
protected Collection> getPlugins() {
@@ -54,6 +56,8 @@ public void testClickBenchQueries() throws IOException {
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
.put("index.refresh_interval", -1)
+ .put("index.replication.type", "SEGMENT")
+ .put("index.optimized.enabled", true)// Enable segment replication for remote store
.build(),
mappings
);
@@ -76,8 +80,8 @@ public void testClickBenchQueries() throws IOException {
XContentParser parser = createParser(JsonXContent.jsonXContent,
sourceFile);
source.parseXContent(parser);
+
SearchResponse response = client().prepareSearch(indexName).setSource(source).get();
- // TODO: Match expected results...
System.out.println(response);
}
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java
index 5b677a49694a2..677b3d3fdfd5d 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java
@@ -230,7 +230,7 @@ public boolean serverSideEncryptionBucketKey() {
* null as the S3 client ignores null header values
*/
public String serverSideEncryptionEncryptionContext() {
- return serverSideEncryptionEncryptionContext.isEmpty()
+ return serverSideEncryptionEncryptionContext == null || serverSideEncryptionEncryptionContext.isEmpty()
? null
: Base64.getEncoder().encodeToString(serverSideEncryptionEncryptionContext.getBytes(StandardCharsets.UTF_8));
}
@@ -239,7 +239,7 @@ public String serverSideEncryptionEncryptionContext() {
* Returns the expected bucket owner if set, else null as the S3 client ignores null header values
*/
public String expectedBucketOwner() {
- return expectedBucketOwner.isEmpty() ? null : expectedBucketOwner;
+ return expectedBucketOwner == null || expectedBucketOwner.isEmpty() ? null : expectedBucketOwner;
}
public long bufferSizeInBytes() {
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java
index 8d8de283f75bb..a8a46065092de 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java
@@ -683,4 +683,10 @@ protected void doClose() {
}
super.doClose();
}
+
+ @Override
+ public boolean isSeverSideEncryptionEnabled() {
+ // s3 is always server side encrypted.
+ return true;
+ }
}
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
index 49c6a31e32816..d75598fb6b782 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
@@ -33,6 +33,7 @@
package org.opensearch.repositories.s3;
import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.ServerSideEncryption;
import org.opensearch.cluster.metadata.RepositoryMetadata;
import org.opensearch.common.blobstore.BlobStoreException;
@@ -175,6 +176,18 @@ public void testValidateHttpLClientType_Invalid_Values() {
}
}
+ public void testIsSeverSideEncryptionEnabled_When_AWSKMS_Type() {
+ Settings settings = Settings.builder()
+ .put(S3Repository.SERVER_SIDE_ENCRYPTION_TYPE_SETTING.getKey(), ServerSideEncryption.AWS_KMS.toString())
+ .build();
+ final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", settings);
+ try (S3Repository s3Repo = createS3Repo(metadata)) {
+
+ // Don't expect any Exception
+ assertTrue(s3Repo.isSeverSideEncryptionEnabled());
+ }
+ }
+
private S3Repository createS3Repo(RepositoryMetadata metadata) {
return new S3Repository(
metadata,
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..c72820d8df19a 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
@@ -198,7 +198,9 @@ private Map> getSegmentMetadata(
IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.get(indexMetadata.getSettings()),
index.getUUID(),
shardId,
- indexSettings.getRemoteStorePathStrategy()
+ indexSettings.getRemoteStorePathStrategy(),
+ null,
+ RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata())
);
Map segmentMetadataMapWithFilenames = remoteDirectory.readLatestNMetadataFiles(5);
@@ -257,7 +259,8 @@ private Map> getTranslogMetadataFiles(
tracker,
indexSettings.getRemoteStorePathStrategy(),
new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()),
- RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata)
+ RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata),
+ RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata())
);
Map metadataMap = manager.readLatestNMetadataFiles(5);
diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java
index 4d53a547db714..95d2e13c6d417 100644
--- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java
+++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java
@@ -992,6 +992,7 @@ public Iterator> settings() {
public static final String KEY_PRIMARY_TERMS = "primary_terms";
public static final String REMOTE_STORE_CUSTOM_KEY = "remote_store";
public static final String TRANSLOG_METADATA_KEY = "translog_metadata";
+ public static final String REMOTE_STORE_SSE_ENABLED_INDEX_KEY = "sse_enabled_index";
public static final String CONTEXT_KEY = "context";
public static final String INGESTION_SOURCE_KEY = "ingestion_source";
public static final String INGESTION_STATUS_KEY = "ingestion_status";
diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
index a889091140d12..7bc8c9ccb0855 100644
--- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
+++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java
@@ -632,7 +632,8 @@ static Optional validateOverlap(Set requestSettings, Settings co
IndexMetadata buildAndValidateTemporaryIndexMetadata(
final Settings aggregatedIndexSettings,
final CreateIndexClusterStateUpdateRequest request,
- final int routingNumShards
+ final int routingNumShards,
+ final ClusterState clusterState
) {
final boolean isHiddenAfterTemplates = IndexMetadata.INDEX_HIDDEN_SETTING.get(aggregatedIndexSettings);
@@ -642,7 +643,7 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata(
tmpImdBuilder.setRoutingNumShards(routingNumShards);
tmpImdBuilder.settings(aggregatedIndexSettings);
tmpImdBuilder.system(isSystem);
- addRemoteStoreCustomMetadata(tmpImdBuilder, true);
+ addRemoteStoreCustomMetadata(tmpImdBuilder, true, clusterState);
if (request.context() != null) {
tmpImdBuilder.context(request.context());
@@ -661,7 +662,9 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata(
* @param tmpImdBuilder index metadata builder.
* @param assertNullOldType flag to verify that the old remote store path type is null
*/
- public void addRemoteStoreCustomMetadata(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType) {
+ public void addRemoteStoreCustomMetadata(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType, ClusterState clusterState) {
+
+ boolean isRestoreFromSnapshot = !assertNullOldType;
if (remoteStoreCustomMetadataResolver == null) {
return;
}
@@ -676,6 +679,24 @@ public void addRemoteStoreCustomMetadata(IndexMetadata.Builder tmpImdBuilder, bo
boolean isTranslogMetadataEnabled = remoteStoreCustomMetadataResolver.isTranslogMetadataEnabled();
remoteCustomData.put(IndexMetadata.TRANSLOG_METADATA_KEY, Boolean.toString(isTranslogMetadataEnabled));
+ Optional remoteNode = clusterState.nodes()
+ .getNodes()
+ .values()
+ .stream()
+ .filter(DiscoveryNode::isRemoteStoreNode)
+ .findFirst();
+
+ String sseEnabledIndex = existingCustomData == null
+ ? null
+ : existingCustomData.get(IndexMetadata.REMOTE_STORE_SSE_ENABLED_INDEX_KEY);
+ if (isRestoreFromSnapshot && sseEnabledIndex != null) {
+ remoteCustomData.put(IndexMetadata.REMOTE_STORE_SSE_ENABLED_INDEX_KEY, sseEnabledIndex);
+ } else if (remoteNode.isPresent()
+ && !isRestoreFromSnapshot
+ && remoteStoreCustomMetadataResolver.isRemoteStoreRepoServerSideEncryptionEnabled()) {
+ remoteCustomData.put(IndexMetadata.REMOTE_STORE_SSE_ENABLED_INDEX_KEY, Boolean.toString(true));
+ }
+
// Determine the path type for use using the remoteStorePathResolver.
RemoteStorePathStrategy newPathStrategy = remoteStoreCustomMetadataResolver.getPathStrategy();
remoteCustomData.put(PathType.NAME, newPathStrategy.getType().name());
@@ -730,7 +751,7 @@ private ClusterState applyCreateIndexRequestWithV1Templates(
clusterService.getClusterSettings()
);
int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null);
- IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards);
+ IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards, currentState);
return applyCreateIndexWithTemporaryService(
currentState,
@@ -795,7 +816,7 @@ private ClusterState applyCreateIndexRequestWithV2Template(
clusterService.getClusterSettings()
);
int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null);
- IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards);
+ IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards, currentState);
return applyCreateIndexWithTemporaryService(
currentState,
@@ -879,7 +900,7 @@ private ClusterState applyCreateIndexRequestWithExistingMetadata(
clusterService.getClusterSettings()
);
final int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, sourceMetadata);
- IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards);
+ IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards, currentState);
return applyCreateIndexWithTemporaryService(
currentState,
@@ -1177,8 +1198,8 @@ public static void updateRemoteStoreSettings(
.findFirst();
if (remoteNode.isPresent()) {
- translogRepo = RemoteStoreNodeAttribute.getTranslogRepoName(remoteNode.get().getAttributes());
segmentRepo = RemoteStoreNodeAttribute.getSegmentRepoName(remoteNode.get().getAttributes());
+ translogRepo = RemoteStoreNodeAttribute.getTranslogRepoName(remoteNode.get().getAttributes());
if (segmentRepo != null) {
settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true).put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepo);
if (translogRepo != null) {
diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java
index 9b33bde32cb54..9efcff47166f0 100644
--- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java
+++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java
@@ -807,6 +807,8 @@ public void apply(Settings value, Settings current, Settings previous) {
RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED,
RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX,
RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX,
+ // Server Side encryption enabled
+ RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_ENABLED,
// Snapshot related Settings
BlobStoreRepository.SNAPSHOT_SHARD_PATH_PREFIX_SETTING,
diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java
index 972b1c54d300f..939f6d2944073 100644
--- a/server/src/main/java/org/opensearch/index/IndexService.java
+++ b/server/src/main/java/org/opensearch/index/IndexService.java
@@ -84,6 +84,7 @@
import org.opensearch.index.query.QueryShardContext;
import org.opensearch.index.query.SearchIndexNameMatcher;
import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory;
+import org.opensearch.index.remote.RemoteStoreUtils;
import org.opensearch.index.seqno.RetentionLeaseSyncer;
import org.opensearch.index.shard.IndexEventListener;
import org.opensearch.index.shard.IndexShard;
@@ -719,7 +720,8 @@ public synchronized IndexShard createShard(
this.indexSettings.getUUID(),
shardId,
this.indexSettings.getRemoteStorePathStrategy(),
- this.indexSettings.getRemoteStoreSegmentPathPrefix()
+ this.indexSettings.getRemoteStoreSegmentPathPrefix(),
+ RemoteStoreUtils.isServerSideEncryptionEnabledIndex(this.indexSettings.getIndexMetadata())
);
}
// When an instance of Store is created, a shardlock is created which is released on closing the instance of store.
@@ -736,7 +738,9 @@ protected void closeInternal() {
// Do nothing for shard lock on remote store
}
};
- CompositeStoreDirectory remoteCompositeStoreDirectory = createCompositeStoreDirectory(path);
+ CompositeStoreDirectory remoteCompositeStoreDirectory = this.indexSettings.isOptimizedIndex()
+ ? createCompositeStoreDirectory(shardId, path)
+ : null;
remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, remoteStoreLock, Store.OnClose.EMPTY, path, remoteCompositeStoreDirectory);
} else {
// Disallow shards with remote store based settings to be created on non-remote store enabled nodes
@@ -767,7 +771,9 @@ protected void closeInternal() {
directory = directoryFactory.newDirectory(this.indexSettings, path);
}
- CompositeStoreDirectory compositeStoreDirectory = createCompositeStoreDirectory(path);
+ CompositeStoreDirectory compositeStoreDirectory = this.indexSettings.isOptimizedIndex()
+ ? createCompositeStoreDirectory(shardId, path)
+ : null;
store = new Store(
shardId,
@@ -1366,11 +1372,12 @@ final IndexStorePlugin.DirectoryFactory getDirectoryFactory() {
* 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 {
+ private CompositeStoreDirectory createCompositeStoreDirectory(ShardId shardId, ShardPath shardPath) throws IOException {
if (compositeStoreDirectoryFactory != null) {
logger.debug("Using CompositeStoreDirectoryFactory to create directory for shard path: {}", shardPath);
return compositeStoreDirectoryFactory.newCompositeStoreDirectory(
indexSettings,
+ shardId,
shardPath,
pluginsService
);
diff --git a/server/src/main/java/org/opensearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/opensearch/index/engine/CombinedDeletionPolicy.java
index 338112745eb54..4589455ab5d6e 100644
--- a/server/src/main/java/org/opensearch/index/engine/CombinedDeletionPolicy.java
+++ b/server/src/main/java/org/opensearch/index/engine/CombinedDeletionPolicy.java
@@ -175,10 +175,15 @@ public SafeCommitInfo getSafeCommitInfo() {
* Index files of the capturing commit point won't be released until the commit reference is closed.
*
* @param acquiringSafeCommit captures the most recent safe commit point if true; otherwise captures the most recent commit point.
+ * @throws EngineNotInitializedException if the deletion policy has not been initialized yet (no commits exist)
*/
public synchronized IndexCommit acquireIndexCommit(boolean acquiringSafeCommit) {
- assert safeCommit != null : "Safe commit is not initialized yet";
- assert lastCommit != null : "Last commit is not initialized yet";
+ if (safeCommit == null) {
+ throw new EngineNotInitializedException("Safe commit is not initialized yet - deletion policy has not processed any commits");
+ }
+ if (lastCommit == null) {
+ throw new EngineNotInitializedException("Last commit is not initialized yet - deletion policy has not processed any commits");
+ }
final IndexCommit snapshotting = acquiringSafeCommit ? safeCommit : lastCommit;
snapshottedCommits.merge(snapshotting, 1, Integer::sum); // increase refCount
return new SnapshotIndexCommit(snapshotting);
diff --git a/server/src/main/java/org/opensearch/index/engine/Engine.java b/server/src/main/java/org/opensearch/index/engine/Engine.java
index f9898382ffbdc..92938e6728192 100644
--- a/server/src/main/java/org/opensearch/index/engine/Engine.java
+++ b/server/src/main/java/org/opensearch/index/engine/Engine.java
@@ -84,6 +84,9 @@
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.engine.exec.coord.SegmentInfosCatalogSnapshot;
import org.opensearch.index.mapper.IdFieldMapper;
import org.opensearch.index.mapper.Mapping;
import org.opensearch.index.mapper.ParseContext.Document;
@@ -301,6 +304,19 @@ public long getMaxSeqNoFromSegmentInfos(SegmentInfos segmentInfos) throws IOExce
}
}
+ @Override
+ public CompositeEngine.ReleasableRef acquireSnapshot() {
+ GatedCloseable segmentInfosCloseable = getSegmentInfosSnapshot();
+ return new CompositeEngine.ReleasableRef(
+ new SegmentInfosCatalogSnapshot(segmentInfosCloseable.get())
+ ) {
+ @Override
+ public void close() throws Exception {
+ segmentInfosCloseable.close();
+ }
+ };
+ }
+
/**
* Get max sequence number that is part of given searcher. Sequence number is part of each document that is indexed.
* This method fetches the _id of last indexed document that was part of the given searcher and
diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java
index 4bfa1cbef71ab..814b5848c5be3 100644
--- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java
+++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java
@@ -559,7 +559,7 @@ public final boolean assertSearcherIsWarmedUp(String source, SearcherScope scope
case "segments_stats":
break;
default:
-// assert externalReaderManager.isWarmedUp : "searcher was not warmed up yet for source[" + source + "]";
+ // assert externalReaderManager.isWarmedUp : "searcher was not warmed up yet for source[" + source + "]";
}
}
return true;
diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationCompositeEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationCompositeEngine.java
new file mode 100644
index 0000000000000..06c19e3786172
--- /dev/null
+++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationCompositeEngine.java
@@ -0,0 +1,466 @@
+/*
+ * 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.engine;
+
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.opensearch.common.concurrent.GatedCloseable;
+import org.opensearch.common.logging.Loggers;
+import org.opensearch.common.util.concurrent.ReleasableLock;
+import org.opensearch.common.util.io.IOUtils;
+import org.opensearch.core.index.shard.ShardId;
+import org.opensearch.index.IndexSettings;
+import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
+import org.opensearch.index.engine.exec.coord.CatalogSnapshotManager;
+import org.opensearch.index.engine.exec.coord.CompositeEngine;
+import org.opensearch.index.engine.exec.coord.CompositeEngineCatalogSnapshot;
+import org.opensearch.index.engine.exec.coord.SegmentInfosCatalogSnapshot;
+import org.opensearch.index.engine.exec.commit.LuceneCommitEngine;
+import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.engine.SearchExecEngine;
+import org.opensearch.index.mapper.MapperService;
+import org.opensearch.index.seqno.LocalCheckpointTracker;
+import org.opensearch.index.seqno.SeqNoStats;
+import org.opensearch.index.seqno.SequenceNumbers;
+import org.opensearch.index.shard.ShardPath;
+import org.opensearch.index.translog.Translog;
+import org.opensearch.index.translog.TranslogCorruptedException;
+import org.opensearch.index.translog.TranslogDeletionPolicy;
+import org.opensearch.index.translog.DefaultTranslogDeletionPolicy;
+import org.opensearch.index.translog.TranslogException;
+import org.opensearch.index.translog.TranslogManager;
+import org.opensearch.index.translog.TranslogOperationHelper;
+import org.opensearch.index.translog.WriteOnlyTranslogManager;
+import org.opensearch.index.translog.listener.TranslogEventListener;
+import org.opensearch.plugins.PluginsService;
+import org.opensearch.plugins.SearchEnginePlugin;
+import org.opensearch.plugins.spi.vectorized.DataFormat;
+import org.opensearch.search.suggest.completion.CompletionStats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.opensearch.index.seqno.SequenceNumbers.MAX_SEQ_NO;
+
+/**
+ * Engine implementation for replica shards with optimized (multi-format) indices.
+ * Combines segment replication behavior (translog-only writes) with multi-format support.
+ *
+ * Similar to NRTReplicationEngine but for optimized indices using CatalogSnapshot.
+ */
+public class NRTReplicationCompositeEngine extends CompositeEngine {
+
+ private final ShardId shardId;
+ private final Logger logger;
+ private final WriteOnlyTranslogManager translogManager;
+ private final LocalCheckpointTracker localCheckpointTracker;
+ private final AtomicBoolean isClosed = new AtomicBoolean(false);
+ private final List refreshListeners = new ArrayList<>();
+ private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock();
+ private final ReleasableLock readLock = new ReleasableLock(rwl.readLock());
+ private final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock());
+ private final Lock flushLock = new ReentrantLock();
+ private final Map>> readEngines = new HashMap<>();
+
+ private volatile long lastReceivedPrimaryGen = SequenceNumbers.NO_OPS_PERFORMED;
+
+ public NRTReplicationCompositeEngine(
+ EngineConfig engineConfig,
+ MapperService mapperService,
+ PluginsService pluginsService,
+ IndexSettings indexSettings,
+ ShardPath shardPath,
+ BiFunction localCheckpointTrackerSupplier,
+ TranslogEventListener translogEventListener
+ ) {
+ super(engineConfig, mapperService, pluginsService, indexSettings, shardPath, localCheckpointTrackerSupplier, translogEventListener);
+ this.shardId = engineConfig.getShardId();
+ this.logger = Loggers.getLogger(NRTReplicationCompositeEngine.class, shardId);
+
+ store.incRef();
+ WriteOnlyTranslogManager translogManagerRef = null;
+ CatalogSnapshotManager catalogSnapshotManagerRef = null;
+ boolean success = false;
+
+ try {
+ // Read last committed segment infos
+ final SegmentInfos lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo();
+ final Map userData = lastCommittedSegmentInfos.getUserData();
+ final String translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY));
+
+ // Initialize local checkpoint tracker
+ final SequenceNumbers.CommitInfo commitInfo = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(userData.entrySet());
+ this.localCheckpointTracker = localCheckpointTrackerSupplier.apply(commitInfo.maxSeqNo, commitInfo.localCheckpoint);
+
+ // Register ONLY internal refresh listeners (not external)
+ for (ReferenceManager.RefreshListener listener : engineConfig.getInternalRefreshListener()) {
+ this.refreshListeners.add(listener);
+ }
+
+ // Create write-only translog manager
+ TranslogEventListener internalTranslogEventListener = new TranslogEventListener() {
+ @Override
+ public void onFailure(String reason, Exception ex) {
+ failEngine(reason, ex);
+ }
+
+ @Override
+ public void onAfterTranslogSync() {
+ try {
+ translogManager.trimUnreferencedReaders();
+ } catch (IOException ex) {
+ throw new TranslogException(shardId, "failed to trim unreferenced translog readers", ex);
+ }
+ }
+ };
+
+ translogManagerRef = new WriteOnlyTranslogManager(
+ engineConfig.getTranslogConfig(),
+ engineConfig.getPrimaryTermSupplier(),
+ engineConfig.getGlobalCheckpointSupplier(),
+ new DefaultTranslogDeletionPolicy(
+ engineConfig.getIndexSettings().getTranslogRetentionSize().getBytes(),
+ engineConfig.getIndexSettings().getTranslogRetentionAge().getMillis(),
+ engineConfig.getIndexSettings().getTranslogRetentionTotalFiles()
+ ),
+ shardId,
+ readLock,
+ this::getLocalCheckpointTracker,
+ translogUUID,
+ internalTranslogEventListener,
+ this,
+ engineConfig.getTranslogFactory(),
+ engineConfig.getStartedPrimarySupplier(),
+ TranslogOperationHelper.create(engineConfig)
+ );
+ this.translogManager = translogManagerRef;
+
+ success = true;
+ } catch (IOException | TranslogCorruptedException e) {
+ throw new EngineCreationFailureException(shardId, "failed to create NRTReplicationCompositeEngine", e);
+ } finally {
+ if (!success) {
+ if (translogManagerRef != null) {
+ try {
+ translogManagerRef.close();
+ } catch (Exception e) {
+ logger.warn("Failed to close translog manager", e);
+ }
+ }
+ // CatalogSnapshotManager doesn't implement Closeable
+ if (isClosed.get() == false) {
+ store.decRef();
+ }
+ }
+ }
+ }
+
+ // Translog-only operations (from NRTReplicationEngine)
+
+ @Override
+ public Engine.IndexResult index(Engine.Index index) throws IOException {
+ ensureOpen();
+ Engine.IndexResult indexResult = new Engine.IndexResult(index.version(), index.primaryTerm(), index.seqNo(), false);
+ final Translog.Location location = translogManager.add(new Translog.Index(index, indexResult));
+ indexResult.setTranslogLocation(location);
+ indexResult.setTook(System.nanoTime() - index.startTime());
+ indexResult.freeze();
+ localCheckpointTracker.advanceMaxSeqNo(index.seqNo());
+ return indexResult;
+ }
+
+ @Override
+ public Engine.DeleteResult delete(Engine.Delete delete) throws IOException {
+ ensureOpen();
+ Engine.DeleteResult deleteResult = new Engine.DeleteResult(delete.version(), delete.primaryTerm(), delete.seqNo(), true);
+ final Translog.Location location = translogManager.add(new Translog.Delete(delete, deleteResult));
+ deleteResult.setTranslogLocation(location);
+ deleteResult.setTook(System.nanoTime() - delete.startTime());
+ deleteResult.freeze();
+ localCheckpointTracker.advanceMaxSeqNo(delete.seqNo());
+ return deleteResult;
+ }
+
+ @Override
+ public Engine.NoOpResult noOp(Engine.NoOp noOp) throws IOException {
+ ensureOpen();
+ Engine.NoOpResult noOpResult = new Engine.NoOpResult(noOp.primaryTerm(), noOp.seqNo());
+ final Translog.Location location = translogManager.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason()));
+ noOpResult.setTranslogLocation(location);
+ noOpResult.setTook(System.nanoTime() - noOp.startTime());
+ noOpResult.freeze();
+ localCheckpointTracker.advanceMaxSeqNo(noOp.seqNo());
+ return noOpResult;
+ }
+
+ /**
+ * Updates segments from primary using CatalogSnapshot.
+ * CRITICAL: Invokes refresh listeners to update replication checkpoint.
+ */
+ public synchronized void updateSegments(final CatalogSnapshot catalogSnapshot) throws IOException {
+ try (ReleasableLock lock = writeLock.acquire()) {
+ ensureOpen();
+
+ final long maxSeqNo = Long.parseLong(catalogSnapshot.getUserData().get(MAX_SEQ_NO));
+ final long incomingGeneration = catalogSnapshot.getGeneration();
+
+ // For replicas, catalog is managed externally - just track the generation
+ // The catalog snapshot is already applied by IndexShard.finalizeReplication()
+
+ // Invoke refresh listeners
+ invokeRefreshListeners(true);
+
+ // Flush if generation changed
+ if (incomingGeneration != this.lastReceivedPrimaryGen) {
+ flush(false, true);
+ translogManager.getDeletionPolicy().setLocalCheckpointOfSafeCommit(maxSeqNo);
+ translogManager.rollTranslogGeneration();
+ }
+
+ this.lastReceivedPrimaryGen = incomingGeneration;
+ localCheckpointTracker.fastForwardProcessedSeqNo(maxSeqNo);
+ }
+ }
+
+ public void finalizeReplication(CatalogSnapshot catalogSnapshot, ShardPath shardPath) throws IOException {
+ catalogSnapshotManager.applyReplicationChanges(catalogSnapshot, shardPath);
+
+ if (catalogSnapshot != null) {
+ long maxGenerationInSnapshot = catalogSnapshot.getLastWriterGeneration();
+ engine.updateWriterGenerationIfNeeded(maxGenerationInSnapshot);
+ }
+
+ updateSearchEngine();
+ updateSegments(catalogSnapshot);
+ }
+
+ private void invokeRefreshListeners(boolean didRefresh) {
+ // Call beforeRefresh
+ refreshListeners.forEach(listener -> {
+ try {
+ listener.beforeRefresh();
+ } catch (IOException e) {
+ logger.error("refresh listener beforeRefresh failed", e);
+ throw new RuntimeException(e);
+ }
+ });
+
+ // Call afterRefresh - ReplicationCheckpointUpdater runs here
+ refreshListeners.forEach(listener -> {
+ try {
+ listener.afterRefresh(didRefresh);
+ } catch (IOException e) {
+ logger.error("refresh listener afterRefresh failed", e);
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void refresh(String source) throws EngineException {
+ // No-op for replicas
+ }
+
+ @Override
+ public TranslogManager translogManager() {
+ return translogManager;
+ }
+
+ @Override
+ public void writeIndexingBuffer() throws EngineException {
+ // No-op
+ }
+
+ @Override
+ public boolean shouldPeriodicallyFlush() {
+ return false;
+ }
+
+ @Override
+ public void flush(boolean force, boolean waitIfOngoing) throws EngineException {
+ ensureOpen();
+ if (engineConfig.getIndexSettings().isWarmIndex()) {
+ return;
+ }
+ try (final ReleasableLock lock = readLock.acquire()) {
+ ensureOpen();
+ if (flushLock.tryLock() == false) {
+ if (waitIfOngoing == false) {
+ return;
+ }
+ flushLock.lock();
+ }
+ try {
+ // For replicas, flush is minimal - just update translog deletion policy
+ translogManager.getDeletionPolicy().setLocalCheckpointOfSafeCommit(
+ localCheckpointTracker.getProcessedCheckpoint()
+ );
+ } catch (Exception e) {
+ maybeFailEngine("flush", e);
+ throw new FlushFailedEngineException(shardId, e);
+ } finally {
+ flushLock.unlock();
+ }
+ }
+ }
+
+ // Checkpoint methods
+
+ @Override
+ public long getPersistedLocalCheckpoint() {
+ return localCheckpointTracker.getPersistedCheckpoint();
+ }
+
+ @Override
+ public long getProcessedLocalCheckpoint() {
+ return localCheckpointTracker.getProcessedCheckpoint();
+ }
+
+ @Override
+ public SeqNoStats getSeqNoStats(long globalCheckpoint) {
+ return localCheckpointTracker.getStats(globalCheckpoint);
+ }
+
+ @Override
+ public long getLastSyncedGlobalCheckpoint() {
+ return translogManager.getLastSyncedGlobalCheckpoint();
+ }
+
+ // Metadata methods
+
+ @Override
+ public long getIndexThrottleTimeInMillis() {
+ return 0;
+ }
+
+ @Override
+ public boolean isThrottled() {
+ return false;
+ }
+
+ @Override
+ public long getIndexBufferRAMBytesUsed() {
+ return 0;
+ }
+
+ // Throttling methods
+
+ @Override
+ public void activateThrottling() {
+ // No-op
+ }
+
+ @Override
+ public void deactivateThrottling() {
+ // No-op
+ }
+
+ // Unsupported operations for replicas
+
+ @Override
+ public void forceMerge(
+ boolean flush,
+ int maxNumSegments,
+ boolean onlyExpungeDeletes,
+ boolean upgrade,
+ boolean upgradeOnlyAncientSegments,
+ String forceMergeUUID
+ ) throws EngineException, IOException {
+ // No-op - replicas don't merge
+ }
+
+ @Override
+ public int fillSeqNoGaps(long primaryTerm) throws IOException {
+ return 0;
+ }
+
+ @Override
+ public SafeCommitInfo getSafeCommitInfo() {
+ return new SafeCommitInfo(localCheckpointTracker.getProcessedCheckpoint(), 0);
+ }
+
+ @Override
+ protected void closeNoLock(String reason, CountDownLatch closedLatch) {
+ if (isClosed.compareAndSet(false, true)) {
+ try {
+ logger.debug("closing NRTReplicationCompositeEngine, reason: {}", reason);
+
+ // Close translog manager
+ if (translogManager != null) {
+ translogManager.close();
+ }
+ store.decRef();
+ } catch (Exception e) {
+ logger.error("failed to close NRTReplicationCompositeEngine", e);
+ }
+ }
+ super.closeNoLock(reason, closedLatch);
+ }
+
+ public LocalCheckpointTracker getLocalCheckpointTracker() {
+ return localCheckpointTracker;
+ }
+
+ @Override
+ public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) {
+ // No-op
+ }
+
+ @Override
+ public long getMaxSeqNoOfUpdatesOrDeletes() {
+ return SequenceNumbers.UNASSIGNED_SEQ_NO;
+ }
+
+ @Override
+ public void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) {
+ // No-op for replicas
+ }
+
+ @Override
+ public Translog.Snapshot newChangesSnapshot(
+ String source,
+ long fromSeqNo,
+ long toSeqNo,
+ boolean requiredFullRange,
+ boolean accurateCount
+ ) throws IOException {
+ throw new UnsupportedOperationException("Not implemented");
+ }
+
+ @Override
+ public int countNumberOfHistoryOperations(String source, long fromSeqNo, long toSeqNumber) throws IOException {
+ return 0;
+ }
+
+ @Override
+ public boolean hasCompleteOperationHistory(String reason, long startingSeqNo) {
+ return false;
+ }
+
+ @Override
+ public long getMinRetainedSeqNo() {
+ return localCheckpointTracker.getProcessedCheckpoint();
+ }
+
+ @Override
+ public Closeable acquireHistoryRetentionLock() {
+ return () -> {};
+ }
+}
diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java
index b97d9931d1139..14f92c7738d45 100644
--- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java
+++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java
@@ -438,7 +438,7 @@ public SafeCommitInfo getSafeCommitInfo() {
}
@Override
- protected final void closeNoLock(String reason, CountDownLatch closedLatch) {
+ protected void closeNoLock(String reason, CountDownLatch closedLatch) {
if (isClosed.compareAndSet(false, true)) {
assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread()
: "Either the write lock must be held or the engine must be currently be failing itself";
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 c1a732707b220..f9e0e85f0b5da 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
@@ -15,6 +15,7 @@
public class FileMetadata {
public static final String DELIMITER = ":::";
+ private static final String METADATA_KEY = "metadata";
private final String file;
private final String dataFormat;
@@ -25,12 +26,16 @@ public FileMetadata(String dataFormat, String file) {
}
public FileMetadata(String dataFormatAwareFile) {
- String[] parts = dataFormatAwareFile.split(DELIMITER);
- if (parts.length != 2) {
- throw new IllegalArgumentException("Expected FileMetadata string to have 2 parts: " + dataFormatAwareFile);
+ if (!dataFormatAwareFile.contains(DELIMITER) && dataFormatAwareFile.startsWith(METADATA_KEY)) {
+ this.dataFormat = "metadata";
+ this.file = dataFormatAwareFile;
+ return;
}
+ String[] parts = dataFormatAwareFile.split(DELIMITER);
+ this.dataFormat = (parts.length == 1)
+ ? "lucene"
+ : parts[1];
this.file = parts[0];
- this.dataFormat = parts[1];
}
public String serialize() {
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/RefreshInput.java b/server/src/main/java/org/opensearch/index/engine/exec/RefreshInput.java
index b772e3ef4ed7a..320847dae9cfc 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/RefreshInput.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/RefreshInput.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
import java.util.ArrayList;
@@ -15,7 +17,7 @@
public class RefreshInput {
- private List existingSegments;
+ private List existingSegments;
private final List writerFiles;
public RefreshInput() {
@@ -23,7 +25,7 @@ public RefreshInput() {
this.existingSegments = new ArrayList<>();
}
- public void setExistingSegments(List existingSegments) {
+ public void setExistingSegments(List existingSegments) {
this.existingSegments = existingSegments;
}
@@ -35,7 +37,7 @@ public List getWriterFiles() {
return writerFiles;
}
- public List getExistingSegments() {
+ public List getExistingSegments() {
return existingSegments;
}
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/RefreshResult.java b/server/src/main/java/org/opensearch/index/engine/exec/RefreshResult.java
index 2df905c49d4bc..809165608b15d 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/RefreshResult.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/RefreshResult.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
import java.util.ArrayList;
@@ -15,17 +17,17 @@
public class RefreshResult {
- private List refreshedSegments;
+ private List refreshedSegments;
public RefreshResult() {
this.refreshedSegments = new ArrayList<>();
}
- public List getRefreshedSegments() {
+ public List getRefreshedSegments() {
return refreshedSegments;
}
- public void setRefreshedSegments(List refreshedSegments) {
+ public void setRefreshedSegments(List refreshedSegments) {
this.refreshedSegments = refreshedSegments;
}
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/WriterFileSet.java b/server/src/main/java/org/opensearch/index/engine/exec/WriterFileSet.java
index 6996ca35305ba..932b12126b5ae 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/WriterFileSet.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/WriterFileSet.java
@@ -60,6 +60,7 @@ public WriterFileSet withDirectory(String newDirectory) {
public void writeTo(StreamOutput out) throws IOException {
out.writeString(directory);
out.writeLong(writerGeneration);
+ out.writeVInt((int) numRows);
out.writeVInt(files.size());
for (String file : files) {
out.writeString(file);
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/bridge/Indexer.java b/server/src/main/java/org/opensearch/index/engine/exec/bridge/Indexer.java
index 46e20f943e860..9559a84f59a6f 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/bridge/Indexer.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/bridge/Indexer.java
@@ -9,17 +9,19 @@
package org.opensearch.index.engine.exec.bridge;
import org.apache.logging.log4j.Logger;
+import org.apache.lucene.index.IndexCommit;
import org.opensearch.ExceptionsHelper;
import org.opensearch.common.Nullable;
import org.opensearch.common.annotation.PublicApi;
+import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.core.common.unit.ByteSizeValue;
-import org.opensearch.index.engine.Engine;
-import org.opensearch.index.engine.EngineException;
-import org.opensearch.index.engine.SafeCommitInfo;
-import org.opensearch.index.engine.Segment;
+import org.opensearch.index.engine.*;
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.seqno.SequenceNumbers;
+import org.opensearch.index.shard.ShardPath;
import org.opensearch.index.translog.Translog;
import org.opensearch.index.translog.TranslogManager;
@@ -31,7 +33,13 @@
import static org.opensearch.index.engine.Engine.HISTORY_UUID_KEY;
@PublicApi(since = "1.0.0")
-public interface Indexer {
+public interface Indexer extends LifecycleAware {
+
+ /**
+ * Returns the engine configuration for this indexer.
+ * @return the engine configuration
+ */
+ EngineConfig config();
/**
* Perform document index operation on the engine
@@ -191,6 +199,18 @@ void forceMerge(
*/
void refresh(String source) throws EngineException;
+ /**
+ * Finalizes replication by applying catalog snapshot changes.
+ * Default no-op implementation for engines that don't support replication.
+ *
+ * @param catalogSnapshot the catalog snapshot to apply
+ * @param shardPath the shard path
+ * @throws IOException if finalization fails
+ */
+ default void finalizeReplication(CatalogSnapshot catalogSnapshot, ShardPath shardPath) throws IOException {
+ // No-op by default
+ }
+
/**
* Commits the data and state to disk, resulting in documents being persisted onto the underlying formats.
*/
@@ -221,6 +241,8 @@ Translog.Snapshot newChangesSnapshot(String source, long fromSeqNo, long toSeqNo
void failEngine(String reason, @Nullable Exception failure);
+ CompositeEngine.ReleasableRef acquireSnapshot();
+
/**
* If the specified throwable contains a fatal error in the throwable graph, such a fatal error will be thrown. Callers should ensure
* that there are no catch statements that would catch an error in the stack as the fatal error here should go uncaught and be handled
@@ -303,6 +325,8 @@ default boolean assertPrimaryIncomingSequenceNumber(final Engine.Operation.Origi
return true;
}
+ GatedCloseable acquireSafeIndexCommit() throws EngineException;
+
/**
* the status of the current doc version in engine, compared to the version in an incoming
* operation
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 6d18035373027..11f25d2e7e2f1 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
@@ -17,6 +17,7 @@
import org.opensearch.common.collect.MapBuilder;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.logging.Loggers;
+import org.opensearch.common.util.io.IOUtils;
import org.opensearch.index.engine.CombinedDeletionPolicy;
import org.opensearch.index.engine.CommitStats;
import org.opensearch.index.engine.EngineException;
@@ -37,12 +38,12 @@
public class LuceneCommitEngine implements Committer {
private final Logger logger;
- private final IndexWriter indexWriter;
+ private IndexWriter indexWriter;
private final CombinedDeletionPolicy combinedDeletionPolicy;
private final Store store;
private volatile SegmentInfos lastCommittedSegmentInfos;
- public LuceneCommitEngine(Store store, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier)
+ public LuceneCommitEngine(Store store, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier, boolean primaryMode)
throws IOException {
this.logger = Loggers.getLogger(LuceneCommitEngine.class, store.shardId());
this.combinedDeletionPolicy = new CombinedDeletionPolicy(logger, translogDeletionPolicy, null, globalCheckpointSupplier);
@@ -50,7 +51,9 @@ public LuceneCommitEngine(Store store, TranslogDeletionPolicy translogDeletionPo
indexWriterConfig.setIndexDeletionPolicy(combinedDeletionPolicy);
this.store = store;
this.lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo();
- this.indexWriter = new IndexWriter(store.directory(), indexWriterConfig);
+ if (primaryMode) {
+ this.indexWriter = new IndexWriter(store.directory(), indexWriterConfig);
+ }
}
@Override
@@ -136,6 +139,6 @@ public GatedCloseable acquireSafeIndexCommit() throws EngineExcepti
@Override
public void close() throws IOException {
- this.indexWriter.close();
+ IOUtils.close(indexWriter);
}
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/composite/CompositeIndexingExecutionEngine.java b/server/src/main/java/org/opensearch/index/engine/exec/composite/CompositeIndexingExecutionEngine.java
index 76b13e90f6437..08603a3401629 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/composite/CompositeIndexingExecutionEngine.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/composite/CompositeIndexingExecutionEngine.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec.composite;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import java.util.Collections;
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicLong;
@@ -79,6 +81,26 @@ public long getNextWriterGeneration() {
return writerGeneration.getAndIncrement();
}
+ /**
+ * Updates the writer generation counter to be at least minGeneration + 1.
+ * This is used during replication/recovery to ensure the replica's writer generation
+ * is always greater than any replicated file's generation, preventing file name collisions.
+ *
+ * @param minGeneration The minimum generation value from replicated files
+ */
+ public void updateWriterGenerationIfNeeded(long minGeneration) {
+ writerGeneration.updateAndGet(current -> Math.max(current, minGeneration + 1));
+ }
+
+ /**
+ * Gets the current writer generation without incrementing.
+ *
+ * @return The current writer generation value
+ */
+ public long getCurrentWriterGeneration() {
+ return writerGeneration.get();
+ }
+
@Override
public List supportedFieldTypes() {
throw new UnsupportedOperationException();
@@ -114,11 +136,11 @@ public RefreshResult refresh(RefreshInput ignore) throws IOException {
RefreshResult finalResult;
try {
List dataFormatWriters = dataFormatWriterPool.checkoutAll();
- List refreshedSegment = ignore.getExistingSegments();
- List newSegmentList = new ArrayList<>();
+ List refreshedSegment = ignore.getExistingSegments();
+ List newSegmentList = new ArrayList<>();
// flush to disk
for (CompositeDataFormatWriter dataFormatWriter : dataFormatWriters) {
- CatalogSnapshot.Segment newSegment = new CatalogSnapshot.Segment(dataFormatWriter.getWriterGeneration());
+ Segment newSegment = new Segment(dataFormatWriter.getWriterGeneration());
FileInfos fileInfos = dataFormatWriter.flush(null);
fileInfos.getWriterFilesMap().forEach((key, value) -> {
newSegment.addSearchableFiles(key.name(), value);
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 2945001caf1c2..2bfcaf5c91396 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
@@ -8,298 +8,73 @@
package org.opensearch.index.engine.exec.coord;
-import org.opensearch.common.annotation.ExperimentalApi;
-import org.opensearch.common.io.stream.BytesStreamOutput;
import org.opensearch.common.util.concurrent.AbstractRefCounted;
-import org.opensearch.core.common.io.stream.*;
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.common.io.stream.StreamOutput;
+import org.opensearch.core.common.io.stream.Writeable;
import org.opensearch.index.engine.exec.FileMetadata;
import org.opensearch.index.engine.exec.WriterFileSet;
-import java.io.*;
+import java.io.IOException;
import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Base64;
import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.function.Supplier;
-@ExperimentalApi
-public class CatalogSnapshot extends AbstractRefCounted implements Writeable, Cloneable {
+public abstract class CatalogSnapshot extends AbstractRefCounted implements Writeable, Cloneable {
+ // Static constants
public static final String CATALOG_SNAPSHOT_KEY = "_catalog_snapshot_";
public static final String LAST_COMPOSITE_WRITER_GEN_KEY = "_last_composite_writer_gen_";
public static final String CATALOG_SNAPSHOT_ID = "_id";
- private final long id;
- private long version;
- private Map userData;
- private long lastWriterGeneration;
- private final Map> dfGroupedSearchableFiles;
- private List segmentList;
- private Supplier indexFileDeleterSupplier;
- private Map catalogSnapshotMap;
- public CatalogSnapshot(long id, long version, List segmentList, Map catalogSnapshotMap, Supplier indexFileDeleterSupplier) {
- super("catalog_snapshot_" + id);
- this.id = id;
- this.segmentList = segmentList;
- this.version = version;
- this.userData = new HashMap<>();
- this.dfGroupedSearchableFiles = new HashMap<>();
- this.lastWriterGeneration = -1;
+ protected final long generation;
+ protected long version;
- segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
- dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
- this.lastWriterGeneration = Math.max(this.lastWriterGeneration, writerFiles.getWriterGeneration());
- }));
- this.catalogSnapshotMap = catalogSnapshotMap;
- this.indexFileDeleterSupplier = indexFileDeleterSupplier;
- // Whenever a new CatalogSnapshot is created add its files to the IndexFileDeleter
- indexFileDeleterSupplier.get().addFileReferences(this);
+ public CatalogSnapshot(String name, long generation, long version) {
+ super(name);
+ this.generation = generation;
+ this.version = version;
}
public CatalogSnapshot(StreamInput in) throws IOException {
super("catalog_snapshot");
- this.id = in.readLong();
+ this.generation = in.readLong();
this.version = in.readLong();
-
- // Read userData map
- int userDataSize = in.readVInt();
- this.userData = new HashMap<>();
- for (int i = 0; i < userDataSize; i++) {
- String key = in.readString();
- String value = in.readString();
- userData.put(key, value);
- }
-
- this.lastWriterGeneration = in.readLong();
-
- int segmentCount = in.readVInt();
- this.segmentList = new ArrayList<>(segmentCount);
- for (int i = 0; i < segmentCount; i++) {
- segmentList.add(new Segment(in));
- }
-
- // Rebuild dfGroupedSearchableFiles from segmentList
- this.dfGroupedSearchableFiles = new HashMap<>();
- segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
- dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
- }));
- }
-
- public void remapPaths(Path newShardDataPath) {
- List remappedSegments = new ArrayList<>();
- for (Segment segment : segmentList) {
- Segment remappedSegment = new Segment(segment.getGeneration());
- for (Map.Entry entry : segment.getDFGroupedSearchableFiles().entrySet()) {
- String dataFormat = entry.getKey();
- // TODO this path resolution should be handled by core components
- Path newDataFormatSpecificShardPath = newShardDataPath.resolve(dataFormat);
- WriterFileSet originalFileSet = entry.getValue();
- WriterFileSet remappedFileSet = originalFileSet.withDirectory(newDataFormatSpecificShardPath.toString());
- remappedSegment.addSearchableFiles(dataFormat, remappedFileSet);
- }
- remappedSegments.add(remappedSegment);
- }
- dfGroupedSearchableFiles.clear();
- this.segmentList = remappedSegments;
- segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
- dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
- }));
}
@Override
public void writeTo(StreamOutput out) throws IOException {
- out.writeLong(id);
+ out.writeLong(generation);
out.writeLong(version);
-
- // Write userData map
- if (userData == null) {
- out.writeVInt(0);
- } else {
- out.writeVInt(userData.size());
- for (Map.Entry entry : userData.entrySet()) {
- out.writeString(entry.getKey());
- out.writeString(entry.getValue());
- }
- }
-
- out.writeLong(lastWriterGeneration);
-
- out.writeVInt(segmentList != null ? segmentList.size() : 0);
- if (segmentList != null) {
- for (Segment segment : segmentList) {
- segment.writeTo(out);
- }
- }
- }
-
- public String serializeToString() throws IOException {
- try (BytesStreamOutput out = new BytesStreamOutput()) {
- this.writeTo(out);
- return Base64.getEncoder().encodeToString(out.bytes().toBytesRef().bytes);
- }
- }
-
- public static CatalogSnapshot deserializeFromString(String serializedData) throws IOException {
- byte[] bytes = Base64.getDecoder().decode(serializedData);
- try (BytesStreamInput in = new BytesStreamInput(bytes)) {
- return new CatalogSnapshot(in);
- }
- }
-
- public Collection getSearchableFiles(String dataFormat) {
- if (dfGroupedSearchableFiles.containsKey(dataFormat)) {
- return dfGroupedSearchableFiles.get(dataFormat);
- }
- return Collections.emptyList();
- }
-
- public List getSegments() {
- return segmentList;
- }
-
- public Collection getFileMetadataList() throws IOException {
- 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,
- fileName
- );
- allFileMetadata.add(fileMetadata);
- }
- });
- }
-
- return allFileMetadata;
}
public long getGeneration() {
- return id;
+ return generation;
}
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 userData;
- }
-
- public void changed() {
- version++;
- }
-
- @Override
- protected void closeInternal() {
- // Notify to FileDeleter to remove references of files referenced in this CatalogSnapshot
- indexFileDeleterSupplier.get().removeFileReferences(this);
- // Remove entry from catalogSnapshotMap
- catalogSnapshotMap.remove(this.id);
- }
-
- public long getId() {
- return id;
- }
-
- public long getLastWriterGeneration() {
- return lastWriterGeneration;
- }
-
- public Set getDataFormats() {
- return dfGroupedSearchableFiles.keySet();
- }
-
- // used only when catalog snapshot is created from last commited segment and hence the object is not initialized with the deleter and map
- public void setIndexFileDeleterSupplier(Supplier supplier) {
- if (this.indexFileDeleterSupplier == null) {
- this.indexFileDeleterSupplier = supplier;
- }
- }
-
- public void setCatalogSnapshotMap(Map catalogSnapshotMap) {
- this.catalogSnapshotMap = catalogSnapshotMap;
- }
-
- @Override
- public String toString() {
- return "CatalogSnapshot{" + "id=" + id + ", version=" + version + ", dfGroupedSearchableFiles=" + dfGroupedSearchableFiles + ", List of Segment= " + segmentList + ", userData=" + userData +'}';
- }
+ // Abstract methods that subclasses must implement
+ public abstract Collection getFileMetadataList() throws IOException;
+ public abstract Map getUserData();
+ public abstract long getId();
+ public abstract List getSegments();
+ public abstract Collection getSearchableFiles(String dataFormat);
+ public abstract Set getDataFormats();
+ public abstract long getLastWriterGeneration();
+ public abstract String serializeToString() throws IOException;
+ public abstract void remapPaths(Path newShardDataPath);
+ public abstract void setIndexFileDeleterSupplier(java.util.function.Supplier supplier);
+ public abstract void setCatalogSnapshotMap(Map catalogSnapshotMap);
public CatalogSnapshot cloneNoAcquire() {
// Still using the clone call since Lucene call requires clone. This will allow a SegmentsInfos backed CatalogSnapshot to use the same method in calls.
return this;
}
- public static class Segment implements Serializable, Writeable {
-
- private final long generation;
- private final Map dfGroupedSearchableFiles;
-
- public Segment(long generation) {
- this.dfGroupedSearchableFiles = new HashMap<>();
- this.generation = generation;
- }
-
- public Segment(StreamInput in) throws IOException {
- this.generation = in.readLong();
- this.dfGroupedSearchableFiles = new HashMap<>();
- int mapSize = in.readVInt();
- for (int i = 0; i < mapSize; i++) {
- String dataFormat = in.readString();
- WriterFileSet writerFileSet = new WriterFileSet(in);
- dfGroupedSearchableFiles.put(dataFormat, writerFileSet);
- }
- }
-
- public void addSearchableFiles(String dataFormat, WriterFileSet writerFileSetGroup) {
- dfGroupedSearchableFiles.put(dataFormat, writerFileSetGroup);
- }
-
- public Map getDFGroupedSearchableFiles() {
- return dfGroupedSearchableFiles;
- }
-
- 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(df , file));
- }
- return searchableFiles;
- }
-
- public long getGeneration() {
- return generation;
- }
-
- @Override
- public void writeTo(StreamOutput out) throws IOException {
- out.writeLong(generation);
- out.writeVInt(dfGroupedSearchableFiles.size());
- for (Map.Entry entry : dfGroupedSearchableFiles.entrySet()) {
- out.writeString(entry.getKey());
- entry.getValue().writeTo(out);
- }
- }
-
- @Override
- public String toString() {
- return "Segment{" + "generation=" + generation + ", dfGroupedSearchableFiles=" + dfGroupedSearchableFiles + '}';
- }
- }
+ public abstract void setUserData(Map userData, boolean b);
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshotManager.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshotManager.java
index a8f5043a2dd53..ebaba77ffdbf7 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshotManager.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/CatalogSnapshotManager.java
@@ -8,6 +8,10 @@
package org.opensearch.index.engine.exec.coord;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.opensearch.index.engine.exec.DataFormat;
import org.opensearch.index.engine.exec.RefreshResult;
import org.opensearch.index.engine.exec.WriterFileSet;
@@ -30,26 +34,32 @@
public class CatalogSnapshotManager {
- private CatalogSnapshot latestCatalogSnapshot;
+ private static final Logger logger = LogManager.getLogger(CatalogSnapshotManager.class);
+
+ private CompositeEngineCatalogSnapshot latestCatalogSnapshot;
private final Committer compositeEngineCommitter;
- private final Map catalogSnapshotMap;
+ private final Map catalogSnapshotMap;
private final AtomicReference indexFileDeleter;
public CatalogSnapshotManager(CompositeEngine compositeEngine, Committer compositeEngineCommitter, ShardPath shardPath) throws IOException {
catalogSnapshotMap = new HashMap<>();
this.compositeEngineCommitter = compositeEngineCommitter;
indexFileDeleter = new AtomicReference<>();
- getLastCommittedCatalogSnapshot().ifPresent(lastCommittedCatalogSnapshot -> {
+
+ Optional lastCommittedOpt = getLastCommittedCatalogSnapshot();
+
+ lastCommittedOpt.ifPresent(lastCommittedCatalogSnapshot -> {
latestCatalogSnapshot = lastCommittedCatalogSnapshot;
catalogSnapshotMap.put(latestCatalogSnapshot.getId(), latestCatalogSnapshot);
latestCatalogSnapshot.remapPaths(shardPath.getDataPath());
});
+
indexFileDeleter.set(new IndexFileDeleter(compositeEngine, latestCatalogSnapshot, shardPath));
if(latestCatalogSnapshot != null) {
latestCatalogSnapshot.setIndexFileDeleterSupplier(indexFileDeleter::get);
latestCatalogSnapshot.setCatalogSnapshotMap(catalogSnapshotMap);
} else {
- latestCatalogSnapshot = new CatalogSnapshot(1, 1, new ArrayList<>(), catalogSnapshotMap, indexFileDeleter::get);
+ latestCatalogSnapshot = new CompositeEngineCatalogSnapshot(1, 1, new ArrayList<>(), catalogSnapshotMap, indexFileDeleter::get);
catalogSnapshotMap.put(latestCatalogSnapshot.getId(), latestCatalogSnapshot);
}
}
@@ -67,21 +77,29 @@ public void close() {
public synchronized void applyRefreshResult(RefreshResult refreshResult) {
commitCatalogSnapshot(
- new CatalogSnapshot(
+ new CompositeEngineCatalogSnapshot(
latestCatalogSnapshot.getId() + 1,
latestCatalogSnapshot.getVersion() + 1,
refreshResult.getRefreshedSegments(),
catalogSnapshotMap,
- indexFileDeleter::get)
+ indexFileDeleter::get
+ )
);
}
public synchronized void applyReplicationChanges(CatalogSnapshot catalogSnapshot, ShardPath shardPath) {
- CatalogSnapshot oldSnapshot = latestCatalogSnapshot;
+ CompositeEngineCatalogSnapshot oldSnapshot = latestCatalogSnapshot;
if (catalogSnapshot != null) {
- catalogSnapshot.incRef();
catalogSnapshot.remapPaths(shardPath.getDataPath());
- latestCatalogSnapshot = catalogSnapshot;
+
+ CompositeEngineCatalogSnapshot newSnapshot = (CompositeEngineCatalogSnapshot) catalogSnapshot;
+
+ newSnapshot.setIndexFileDeleterSupplier(indexFileDeleter::get);
+ newSnapshot.setCatalogSnapshotMap(catalogSnapshotMap);
+
+ indexFileDeleter.get().addFileReferences(newSnapshot);
+
+ latestCatalogSnapshot = newSnapshot;
catalogSnapshotMap.put(latestCatalogSnapshot.getId(), latestCatalogSnapshot);
}
if (oldSnapshot != null) {
@@ -91,16 +109,16 @@ public synchronized void applyReplicationChanges(CatalogSnapshot catalogSnapshot
public synchronized void applyMergeResults(MergeResult mergeResult, OneMerge oneMerge) {
- List segmentList = latestCatalogSnapshot.getSegments();
+ List segmentList = new ArrayList<>(latestCatalogSnapshot.getSegments());
- CatalogSnapshot.Segment segmentToAdd = getSegment(mergeResult.getMergedWriterFileSet());
- Set segmentsToRemove = new HashSet<>(oneMerge.getSegmentsToMerge());
+ Segment segmentToAdd = getSegment(mergeResult.getMergedWriterFileSet());
+ Set segmentsToRemove = new HashSet<>(oneMerge.getSegmentsToMerge());
boolean inserted = false;
int newSegIdx = 0;
for (int segIdx = 0, cnt = segmentList.size(); segIdx < cnt; segIdx++) {
assert segIdx >= newSegIdx;
- CatalogSnapshot.Segment currSegment = segmentList.get(segIdx);
+ Segment currSegment = segmentList.get(segIdx);
if(segmentsToRemove.contains(currSegment)) {
if (!inserted) {
segmentList.set(segIdx, segmentToAdd);
@@ -124,13 +142,13 @@ public synchronized void applyMergeResults(MergeResult mergeResult, OneMerge one
if (!inserted) {
segmentList.add(0, segmentToAdd);
}
- CatalogSnapshot newCatSnap = new CatalogSnapshot(latestCatalogSnapshot.getId() + 1, latestCatalogSnapshot.getVersion() + 1, segmentList, catalogSnapshotMap, indexFileDeleter::get);
+ CompositeEngineCatalogSnapshot newCatSnap = new CompositeEngineCatalogSnapshot(latestCatalogSnapshot.getId() + 1, latestCatalogSnapshot.getVersion() + 1, segmentList, catalogSnapshotMap, indexFileDeleter::get);
// Commit new catalog snapshot
commitCatalogSnapshot(newCatSnap);
}
- private synchronized void commitCatalogSnapshot(CatalogSnapshot newCatSnap) {
+ private synchronized void commitCatalogSnapshot(CompositeEngineCatalogSnapshot newCatSnap) {
catalogSnapshotMap.put(newCatSnap.getId(), newCatSnap);
if (latestCatalogSnapshot != null) {
latestCatalogSnapshot.decRef();
@@ -139,8 +157,8 @@ private synchronized void commitCatalogSnapshot(CatalogSnapshot newCatSnap) {
compositeEngineCommitter.addLuceneIndexes(latestCatalogSnapshot);
}
- private CatalogSnapshot.Segment getSegment(Map writerFileSetMap) {
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(0);
+ private Segment getSegment(Map writerFileSetMap) {
+ Segment segment = new Segment(0);
for(DataFormat dataFormat : writerFileSetMap.keySet()) {
segment.addSearchableFiles(dataFormat.name(), writerFileSetMap.get(dataFormat));
@@ -148,11 +166,15 @@ private CatalogSnapshot.Segment getSegment(Map writer
return segment;
}
- private Optional getLastCommittedCatalogSnapshot() throws IOException {
+ private Optional getLastCommittedCatalogSnapshot() throws IOException {
Map lastCommittedData = compositeEngineCommitter.getLastCommittedData();
+
if (lastCommittedData.containsKey(CATALOG_SNAPSHOT_KEY)) {
- return Optional.of(CatalogSnapshot.deserializeFromString(lastCommittedData.get(CATALOG_SNAPSHOT_KEY)));
+ String serializedSnapshot = lastCommittedData.get(CATALOG_SNAPSHOT_KEY);
+ CompositeEngineCatalogSnapshot snapshot = CompositeEngineCatalogSnapshot.deserializeFromString(serializedSnapshot);
+ return Optional.of(snapshot);
}
+
return Optional.empty();
}
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 ab731ba79863f..207c89213d60b 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
@@ -10,11 +10,13 @@
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
+import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.common.Nullable;
import org.opensearch.common.SetOnce;
+import org.opensearch.common.TriConsumer;
import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.lease.Releasable;
@@ -47,6 +49,7 @@
import org.opensearch.index.engine.Segment;
import org.opensearch.index.engine.SegmentsStats;
import org.opensearch.index.engine.VersionValue;
+import org.opensearch.index.engine.exec.FileMetadata;
import org.opensearch.index.engine.exec.FileStats;
import org.opensearch.index.engine.exec.RefreshInput;
import org.opensearch.index.engine.exec.RefreshResult;
@@ -74,12 +77,14 @@
import org.opensearch.index.shard.DocsStats;
import org.opensearch.index.shard.ShardPath;
import org.opensearch.index.store.Store;
+import org.opensearch.index.translog.Checkpoint;
import org.opensearch.index.translog.DefaultTranslogDeletionPolicy;
import org.opensearch.index.translog.InternalTranslogManager;
import org.opensearch.index.translog.Translog;
import org.opensearch.index.translog.TranslogCorruptedException;
import org.opensearch.index.translog.TranslogDeletionPolicy;
import org.opensearch.index.translog.TranslogException;
+import org.opensearch.index.translog.TranslogHeader;
import org.opensearch.index.translog.TranslogManager;
import org.opensearch.index.translog.TranslogOperationHelper;
import org.opensearch.index.translog.listener.CompositeTranslogEventListener;
@@ -87,15 +92,15 @@
import org.opensearch.indices.pollingingest.PollingIngestStats;
import org.opensearch.plugins.PluginsService;
import org.opensearch.plugins.SearchEnginePlugin;
-import org.opensearch.plugins.spi.vectorized.DataFormat;
import org.opensearch.search.suggest.completion.CompletionStats;
+import org.opensearch.plugins.spi.vectorized.DataFormat;
import java.io.Closeable;
import java.io.IOException;
+import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -109,7 +114,6 @@
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
@@ -118,13 +122,13 @@
import static org.opensearch.index.engine.Engine.HISTORY_UUID_KEY;
import static org.opensearch.index.engine.Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID;
-import static org.opensearch.index.engine.exec.coord.CatalogSnapshot.CATALOG_SNAPSHOT_ID;
import static org.opensearch.index.engine.exec.coord.CatalogSnapshot.CATALOG_SNAPSHOT_KEY;
import static org.opensearch.index.engine.exec.coord.CatalogSnapshot.LAST_COMPOSITE_WRITER_GEN_KEY;
@ExperimentalApi
public class CompositeEngine implements LifecycleAware, Closeable, Indexer, CheckpointState, IndexingThrottler, StatsHolder {
+
private static final Consumer PRE_REFRESH_LISTENER_CONSUMER = refreshListener -> {
try {
refreshListener.beforeRefresh();
@@ -139,10 +143,11 @@ public class CompositeEngine implements LifecycleAware, Closeable, Indexer, Chec
throw new RuntimeException(e);
}
};
- private static final BiConsumer>, CatalogSnapshotAwareRefreshListener>
- POST_REFRESH_CATALOG_SNAPSHOT_AWARE_LISTENER_CONSUMER = (catalogSnapshot, catalogSnapshotAwareRefreshListener) -> {
+ private static final TriConsumer>, CatalogSnapshotAwareRefreshListener, Boolean>
+ POST_REFRESH_CATALOG_SNAPSHOT_AWARE_LISTENER_CONSUMER = (catalogSnapshot, catalogSnapshotAwareRefreshListener, didRefresh) -> {
try {
- catalogSnapshotAwareRefreshListener.afterRefresh(true, catalogSnapshot);
+ // Wrap in Supplier as required by CatalogSnapshotAwareRefreshListener interface
+ catalogSnapshotAwareRefreshListener.afterRefresh(didRefresh, catalogSnapshot);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -150,9 +155,9 @@ public class CompositeEngine implements LifecycleAware, Closeable, Indexer, Chec
private static final Function extractSegmentName = name -> name.substring(name.lastIndexOf('_'), name.lastIndexOf('.'));
private final ShardId shardId;
- private final CompositeIndexingExecutionEngine engine;
- private final EngineConfig engineConfig;
- private final Store store;
+ protected final CompositeIndexingExecutionEngine engine;
+ protected final EngineConfig engineConfig;
+ protected final Store store;
private final Logger logger;
private final Committer compositeEngineCommitter;
private final TranslogManager translogManager;
@@ -200,7 +205,7 @@ public class CompositeEngine implements LifecycleAware, Closeable, Indexer, Chec
// The value of this marker never goes backwards, and is tracked/updated differently on primary and replica.
private final AtomicLong maxSeqNoOfUpdatesOrDeletes;
private final IndexingStrategyPlanner indexingStrategyPlanner;
- private final CatalogSnapshotManager catalogSnapshotManager;
+ protected final CatalogSnapshotManager catalogSnapshotManager;
private ReleasableRef lastCommitedCatalogSnapshotRef;
private final EventListener eventListener;
@@ -231,9 +236,30 @@ public CompositeEngine(
this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier);
this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker);
refreshListeners.add(lastRefreshedCheckpointListener);
-
- final Map userData = store.readLastCommittedSegmentsInfo().getUserData();
- String translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY));
+ Map userData;
+ String translogUUID;
+ // Note: lastRefreshedCheckpointListener is initialized later after localCheckpointTracker is ready
+ try {
+ final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo();
+ userData = segmentInfos.getUserData();
+ translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY));
+ } catch (java.io.FileNotFoundException e) {
+ // Local store is empty (remote store recovery scenario)
+ final Path translogPath = engineConfig.getTranslogConfig().getTranslogPath();
+ final Checkpoint checkpoint = Checkpoint.read(translogPath.resolve(Translog.CHECKPOINT_FILE_NAME));
+ final Path translogFile = translogPath.resolve(Translog.getFilename(checkpoint.getGeneration()));
+ try (java.nio.channels.FileChannel channel = java.nio.channels.FileChannel.open(translogFile, java.nio.file.StandardOpenOption.READ)) {
+ final TranslogHeader translogHeader = TranslogHeader.read(translogFile, channel);
+ translogUUID = translogHeader.getTranslogUUID();
+
+ // Create initial empty commit for LuceneCommitEngine
+ store.createEmpty(engineConfig.getIndexSettings().getIndexVersionCreated().luceneVersion, translogUUID);
+
+ // Now read the userData from the newly created commit
+ userData = store.readLastCommittedSegmentsInfo().getUserData();
+ logger.debug("Created initial empty commit with translog UUID: {}", translogUUID);
+ }
+ }
TranslogEventListener internalTranslogEventListener = new TranslogEventListener() {
@Override
public void onAfterTranslogSync() {
@@ -265,7 +291,7 @@ public void onFailure(String reason, Exception ex) {
this.translogManager = translogManagerRef;
// initialize committer and composite indexing execution engine
- committerRef = new LuceneCommitEngine(store, translogDeletionPolicy, translogManager::getLastSyncedGlobalCheckpoint);
+ committerRef = new LuceneCommitEngine(store, translogDeletionPolicy, translogManager::getLastSyncedGlobalCheckpoint, !config().isReadOnlyReplica());
this.compositeEngineCommitter = committerRef;
final AtomicLong lastCommittedWriterGeneration = new AtomicLong(-1);
Map lastCommittedData = this.compositeEngineCommitter.getLastCommittedData();
@@ -285,7 +311,13 @@ public void onFailure(String reason, Exception ex) {
//Initialize CatalogSnapshotManager before loadWriterFiles to ensure stale files are cleaned up before loading
this.catalogSnapshotManager = new CatalogSnapshotManager(this, committerRef, shardPath);
try (CompositeEngine.ReleasableRef catalogSnapshotReleasableRef = catalogSnapshotManager.acquireSnapshot()) {
- this.engine.loadWriterFiles(catalogSnapshotReleasableRef.getRef());
+ CatalogSnapshot loadedSnapshot = catalogSnapshotReleasableRef.getRef();
+ this.engine.loadWriterFiles(loadedSnapshot);
+
+ if (loadedSnapshot != null) {
+ long snapshotLastWriterGen = loadedSnapshot.getLastWriterGeneration();
+ engine.updateWriterGenerationIfNeeded(snapshotLastWriterGen);
+ }
} catch (Exception e) {
failEngine("unable to close releasable catalog snapshot while bootstrapping composite engine", e);
}
@@ -313,6 +345,7 @@ public void onFailure(String reason, Exception ex) {
// Refresh here so that catalog snapshot gets initialized
// TODO : any better way to do this ?
+ initializeRefreshListeners(engineConfig);
refresh("start");
// TODO : how to extend this for Lucene ? where engine is a r/w engine
// Create read specific engines for each format which is associated with shard
@@ -320,8 +353,20 @@ public void onFailure(String reason, Exception ex) {
for (SearchEnginePlugin searchEnginePlugin : searchEnginePlugins) {
for (DataFormat dataFormat : searchEnginePlugin.getSupportedFormats()) {
List> currentSearchEngines = readEngines.getOrDefault(dataFormat, new ArrayList<>());
+
+ // Get FileMetadata filtered by data format from current catalog snapshot
+ Collection formatFiles;
+ try (ReleasableRef snapshotRef = acquireSnapshot()) {
+ CatalogSnapshot snapshot = snapshotRef.getRef();
+ formatFiles = snapshot.getFileMetadataList().stream()
+ .filter(fm -> fm.dataFormat().equals(dataFormat.getName()))
+ .collect(Collectors.toList());
+ } catch (Exception e) {
+ throw new EngineCreationFailureException(shardId, "failed to acquire catalog snapshot for read engine creation", e);
+ }
+
SearchExecEngine, ?, ?, ?> newSearchEngine =
- searchEnginePlugin.createEngine(dataFormat, Collections.emptyList(), shardPath);
+ searchEnginePlugin.createEngine(dataFormat, formatFiles, shardPath);
currentSearchEngines.add(newSearchEngine);
readEngines.put(dataFormat, currentSearchEngines);
@@ -342,10 +387,7 @@ public void onFailure(String reason, Exception ex) {
}
}
}
- catalogSnapshotAwareRefreshListeners.forEach(refreshListener -> POST_REFRESH_CATALOG_SNAPSHOT_AWARE_LISTENER_CONSUMER.accept(
- this::acquireSnapshot,
- refreshListener
- ));
+ invokeRefreshListeners(true);
success = true;
} catch (IOException | TranslogCorruptedException e) {
throw new EngineCreationFailureException(shardId, "failed to create engine", e);
@@ -359,9 +401,6 @@ public void onFailure(String reason, Exception ex) {
}
}
logger.trace("created new CompositeEngine");
-
- initializeRefreshListeners(engineConfig);
-
}
private LocalCheckpointTracker createLocalCheckpointTracker(
@@ -369,11 +408,26 @@ private LocalCheckpointTracker createLocalCheckpointTracker(
) throws IOException {
final long maxSeqNo;
final long localCheckpoint;
- final SequenceNumbers.CommitInfo seqNoStats =
- SequenceNumbers.loadSeqNoInfoFromLuceneCommit(store.readLastCommittedSegmentsInfo().getUserData().entrySet());
- maxSeqNo = seqNoStats.maxSeqNo;
- localCheckpoint = seqNoStats.localCheckpoint;
- logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint);
+
+ try {
+ final SequenceNumbers.CommitInfo seqNoStats =
+ SequenceNumbers.loadSeqNoInfoFromLuceneCommit(store.readLastCommittedSegmentsInfo().getUserData().entrySet());
+ maxSeqNo = seqNoStats.maxSeqNo;
+ localCheckpoint = seqNoStats.localCheckpoint;
+ logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint);
+ } catch (org.apache.lucene.index.IndexNotFoundException e) {
+ // Local store is empty (remote store recovery scenario)
+ // Initialize with NO_OPS_PERFORMED (-1) - checkpoint will be restored from CatalogSnapshot during first flush
+ logger.debug(
+ "Local store is empty during engine initialization, initializing checkpoint tracker with NO_OPS_PERFORMED. "
+ + "This is expected during remote store recovery where local store has not been initialized yet."
+ );
+ return localCheckpointTrackerSupplier.apply(
+ SequenceNumbers.NO_OPS_PERFORMED,
+ SequenceNumbers.NO_OPS_PERFORMED
+ );
+ }
+
return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint);
}
@@ -392,6 +446,11 @@ protected TranslogDeletionPolicy getTranslogDeletionPolicy(EngineConfig engineCo
);
}
+ public final EngineConfig config()
+ {
+ return engineConfig;
+ }
+
protected TranslogManager createTranslogManager(
String translogUUID,
TranslogDeletionPolicy translogDeletionPolicy,
@@ -421,12 +480,13 @@ public void ensureOpen() {
}
}
- LocalCheckpointTracker getLocalCheckpointTracker() {
+ public LocalCheckpointTracker getLocalCheckpointTracker() {
return localCheckpointTracker;
}
+
public void updateSearchEngine() throws IOException {
- catalogSnapshotAwareRefreshListeners.forEach(ref -> {
+ catalogSnapshotAwareRefreshListeners.forEach(ref -> {
try {
ref.afterRefresh(true, catalogSnapshotManager::acquireSnapshot);
} catch (IOException e) {
@@ -705,23 +765,25 @@ public void deactivateThrottling() {
}
public synchronized void refresh(String source) throws EngineException {
+ final long localCheckpointBeforeRefresh = localCheckpointTracker.getProcessedCheckpoint();
+ boolean refreshed = false;
try (CompositeEngine.ReleasableRef catalogSnapshotReleasableRef = catalogSnapshotManager.acquireSnapshot()) {
refreshListeners.forEach(PRE_REFRESH_LISTENER_CONSUMER);
RefreshInput refreshInput = new RefreshInput();
refreshInput.setExistingSegments(new ArrayList<>(catalogSnapshotReleasableRef.getRef().getSegments()));
RefreshResult refreshResult = engine.refresh(refreshInput);
- if (refreshResult == null) {
- return;
+ if (refreshResult != null) {
+ catalogSnapshotManager.applyRefreshResult(refreshResult);
+ refreshed = true;
+ }
+
+ invokeRefreshListeners(refreshed);
+
+ // Call checkpoint listener's afterRefresh to update refreshed checkpoint
+ if (refreshed) {
+ triggerPossibleMerges(); // trigger merges
}
- catalogSnapshotManager.applyRefreshResult(refreshResult);
- catalogSnapshotAwareRefreshListeners.forEach(refreshListener -> POST_REFRESH_CATALOG_SNAPSHOT_AWARE_LISTENER_CONSUMER.accept(
- this::acquireSnapshot,
- refreshListener
- ));
-
- refreshListeners.forEach(POST_REFRESH_LISTENER_CONSUMER);
- triggerPossibleMerges(); // trigger merges
} catch (Exception ex) {
try {
failEngine("refresh failed source[" + source + "]", ex);
@@ -730,11 +792,28 @@ public synchronized void refresh(String source) throws EngineException {
}
throw new RefreshFailedEngineException(shardId, ex);
}
+
+ assert refreshed == false || lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; "
+ + "local_checkpoint="
+ + localCheckpointBeforeRefresh
+ + " refresh_checkpoint="
+ + lastRefreshedCheckpoint();
+ }
+
+ private void invokeRefreshListeners(boolean didRefresh) {
+ catalogSnapshotAwareRefreshListeners.forEach(refreshListener -> POST_REFRESH_CATALOG_SNAPSHOT_AWARE_LISTENER_CONSUMER.apply(
+ this::acquireSnapshot,
+ refreshListener,
+ didRefresh
+ ));
+
+ refreshListeners.forEach(POST_REFRESH_LISTENER_CONSUMER);
}
public synchronized void applyMergeChanges(MergeResult mergeResult, OneMerge oneMerge) {
try {
catalogSnapshotManager.applyMergeResults(mergeResult, oneMerge);
+ invokeRefreshListeners(true);
} catch (Exception ex) {
try {
logger.error(
@@ -755,11 +834,6 @@ public void triggerPossibleMerges() {
mergeScheduler.triggerMerges();
}
- public void finalizeReplication(CatalogSnapshot catalogSnapshot, ShardPath shardPath) throws IOException {
- catalogSnapshotManager.applyReplicationChanges(catalogSnapshot, shardPath);
- updateSearchEngine();
- }
-
// 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
@@ -832,7 +906,7 @@ public List segments(boolean verbose) {
lastCommitedCatalogSnapshotRef.getRef()
.getSegments()
.stream()
- .map(CatalogSnapshot.Segment::getGeneration)
+ .map(org.opensearch.index.engine.exec.coord.Segment::getGeneration)
.collect(Collectors.toCollection(() -> committedSegments));
}
Map segmentStats = getPrimaryReadEngine().fetchSegmentStats();
@@ -909,33 +983,45 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException {
boolean shouldPeriodicallyFlush = shouldPeriodicallyFlush();
if (force || shouldFlush() || shouldPeriodicallyFlush || getProcessedLocalCheckpoint() > Long.parseLong(
readLastCommittedData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))) {
+
translogManager.ensureCanFlush();
+
try {
translogManager.rollTranslogGeneration();
logger.trace("starting commit for flush; commitTranslog=true");
CompositeEngine.ReleasableRef catalogSnapshotToFlushRef = catalogSnapshotManager.acquireSnapshot();
final CatalogSnapshot catalogSnapshotToFlush = catalogSnapshotToFlushRef.getRef();
- System.out.println("FLUSH called, current snapshot to commit : " + catalogSnapshotToFlush.getId()
- + ", previous commited snapshot : " + ((lastCommitedCatalogSnapshotRef != null)
- ? lastCommitedCatalogSnapshotRef.getRef().getId()
- : -1));
- final String serializedCatalogSnapshot = catalogSnapshotToFlush.serializeToString();
- final long lastWriterGeneration = catalogSnapshotToFlush.getLastWriterGeneration();
+
+ // FIX: Use MAX of engine's current counter and snapshot's lastWriterGeneration
+ // to ensure we never reuse a generation after restart.
+ // Engine counter - 1 = last assigned generation (counter points to NEXT generation)
+ final long engineLastAssignedGen = engine.getCurrentWriterGeneration() - 1;
+ final long snapshotLastWriterGen = catalogSnapshotToFlush.getLastWriterGeneration();
+ final long lastWriterGeneration = Math.max(engineLastAssignedGen, snapshotLastWriterGen);
+
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
- final long id = catalogSnapshotToFlush.getId();
+
+ // Create commitData with checkpoint information BEFORE serializing CatalogSnapshot
+ // This ensures CatalogSnapshot.userData contains the correct checkpoint values
+ final Map commitData = new HashMap<>(7);
+ commitData.put(Translog.TRANSLOG_UUID_KEY, translogManager.getTranslogUUID());
+ commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint));
+ commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo()));
+ commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get()));
+ commitData.put(HISTORY_UUID_KEY, historyUUID);
+ commitData.put(LAST_COMPOSITE_WRITER_GEN_KEY, Long.toString(lastWriterGeneration));
+
+ // Copy checkpoint data to CatalogSnapshot.userData BEFORE serialization
+ // This preserves checkpoint state for recovery scenarios (e.g., replica promotion)
+ catalogSnapshotToFlush.setUserData(commitData, false);
+
+ // Now serialize CatalogSnapshot with checkpoint data in userData
+ final String serializedCatalogSnapshot = catalogSnapshotToFlush.serializeToString();
+ commitData.put(CATALOG_SNAPSHOT_KEY, serializedCatalogSnapshot);
+
compositeEngineCommitter.commit(
- () -> {
- final Map commitData = new HashMap<>(7);
- commitData.put(CATALOG_SNAPSHOT_ID, Long.toString(id));
- commitData.put(Translog.TRANSLOG_UUID_KEY, translogManager.getTranslogUUID());
- commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint));
- commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo()));
- commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get()));
- commitData.put(HISTORY_UUID_KEY, historyUUID);
- commitData.put(CATALOG_SNAPSHOT_KEY, serializedCatalogSnapshot);
- commitData.put(LAST_COMPOSITE_WRITER_GEN_KEY, Long.toString(lastWriterGeneration));
- return commitData.entrySet().iterator();
- }, catalogSnapshotToFlush
+ () -> commitData.entrySet().iterator(),
+ catalogSnapshotToFlush
);
logger.trace("finished commit for flush");
if (lastCommitedCatalogSnapshotRef != null && lastCommitedCatalogSnapshotRef.getRef() != null)
@@ -1065,7 +1151,7 @@ public Translog.Snapshot newChangesSnapshot(
boolean requiredFullRange,
boolean accurateCount
) throws IOException {
- return null;
+ return translogManager.newChangesSnapshot(fromSeqNo, toSeqNo, requiredFullRange);
}
@Override
@@ -1127,7 +1213,7 @@ private boolean failOnTragicEvent(AlreadyClosedException ex) {
return engineFailed;
}
- private boolean maybeFailEngine(String source, Exception e) {
+ protected boolean maybeFailEngine(String source, Exception e) {
// Check for AlreadyClosedException -- ACE is a very special
// exception that should only be thrown in a tragic event. we pass on the checks to failOnTragicEvent which will
// throw and AssertionError if the tragic event condition is not met.
@@ -1206,34 +1292,33 @@ private void awaitPendingClose() {
* called while the write lock is hold or in a disaster condition ie. if the engine
* is failed.
*/
- private void closeNoLock(String reason, CountDownLatch closedLatch) {
+ protected void closeNoLock(String reason, CountDownLatch closedLatch) {
if (isClosed.compareAndSet(false, true)) {
assert rwl.isWriteLockedByCurrentThread()
|| failEngineLock.isHeldByCurrentThread() : "Either the write lock must be held or the engine must be currently be failing itself";
try {
- try {
IOUtils.close(engine, translogManager, compositeEngineCommitter);
} catch (Exception e) {
logger.warn("Failed to close translog", e);
- }
- } catch (Exception e) {
- logger.warn("failed to close translog manager", e);
- } finally {
- try {
- store.decRef();
- logger.debug("engine closed [{}]", reason);
} finally {
- closedLatch.countDown();
+ try {
+ store.decRef();
+ logger.debug("engine closed [{}]", reason);
+ } finally {
+ closedLatch.countDown();
+ }
}
- }
}
}
+
+
/**
* Acquires the most recent safe index commit snapshot from the currently running engine.
* All index files referenced by this commit won't be freed until the commit/snapshot is closed.
* This method is required for replica recovery operations.
*/
+ @Override
public GatedCloseable acquireSafeIndexCommit() throws EngineException {
ensureOpen();
if (compositeEngineCommitter instanceof LuceneCommitEngine) {
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngineCatalogSnapshot.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngineCatalogSnapshot.java
new file mode 100644
index 0000000000000..262263d653372
--- /dev/null
+++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/CompositeEngineCatalogSnapshot.java
@@ -0,0 +1,250 @@
+/*
+ * 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.engine.exec.coord;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.opensearch.common.annotation.ExperimentalApi;
+import org.opensearch.common.io.stream.BytesStreamOutput;
+import org.opensearch.core.common.io.stream.*;
+import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.engine.exec.WriterFileSet;
+
+import java.io.*;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Supplier;
+
+@ExperimentalApi
+public class CompositeEngineCatalogSnapshot extends CatalogSnapshot {
+
+ private static final Logger logger = LogManager.getLogger(CompositeEngineCatalogSnapshot.class);
+
+ public static final String CATALOG_SNAPSHOT_KEY = "_catalog_snapshot_";
+ public static final String LAST_COMPOSITE_WRITER_GEN_KEY = "_last_composite_writer_gen_";
+ private Map userData;
+ private long lastWriterGeneration;
+ private final Map> dfGroupedSearchableFiles;
+ private List segmentList;
+ private Supplier indexFileDeleterSupplier;
+ private Map catalogSnapshotMap;
+
+ public CompositeEngineCatalogSnapshot(long id, long version, List segmentList, Map catalogSnapshotMap, Supplier indexFileDeleterSupplier) {
+ super("catalog_snapshot_" + id, id, version);
+ this.segmentList = segmentList;
+ this.userData = new HashMap<>();
+ this.dfGroupedSearchableFiles = new HashMap<>();
+ this.lastWriterGeneration = -1;
+
+ segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
+ dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
+ this.lastWriterGeneration = Math.max(this.lastWriterGeneration, writerFiles.getWriterGeneration());
+ }));
+ this.catalogSnapshotMap = catalogSnapshotMap;
+ this.indexFileDeleterSupplier = indexFileDeleterSupplier;
+ // Whenever a new CatalogSnapshot is created add its files to the IndexFileDeleter
+ indexFileDeleterSupplier.get().addFileReferences(this);
+ }
+
+ public CompositeEngineCatalogSnapshot(StreamInput in) throws IOException {
+ super(in);
+
+ // Read userData map
+ int userDataSize = in.readVInt();
+ this.userData = new HashMap<>();
+ for (int i = 0; i < userDataSize; i++) {
+ String key = in.readString();
+ String value = in.readString();
+ userData.put(key, value);
+ }
+
+ this.lastWriterGeneration = in.readLong();
+
+ int segmentCount = in.readVInt();
+ this.segmentList = new ArrayList<>(segmentCount);
+ for (int i = 0; i < segmentCount; i++) {
+ segmentList.add(new Segment(in));
+ }
+
+ // Rebuild dfGroupedSearchableFiles from segmentList
+ this.dfGroupedSearchableFiles = new HashMap<>();
+ segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
+ dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
+ }));
+ }
+
+ public void remapPaths(Path newShardDataPath) {
+ List remappedSegments = new ArrayList<>();
+ for (Segment segment : segmentList) {
+ Segment remappedSegment = new Segment(segment.getGeneration());
+ for (Map.Entry entry : segment.getDFGroupedSearchableFiles().entrySet()) {
+ String dataFormat = entry.getKey();
+ // TODO this path resolution should be handled by core components
+ Path newDataFormatSpecificShardPath = newShardDataPath.resolve(dataFormat);
+ WriterFileSet originalFileSet = entry.getValue();
+ WriterFileSet remappedFileSet = originalFileSet.withDirectory(newDataFormatSpecificShardPath.toString());
+ remappedSegment.addSearchableFiles(dataFormat, remappedFileSet);
+ }
+ remappedSegments.add(remappedSegment);
+ }
+ dfGroupedSearchableFiles.clear();
+ this.segmentList = remappedSegments;
+ segmentList.forEach(segment -> segment.getDFGroupedSearchableFiles().forEach((dataFormat, writerFiles) -> {
+ dfGroupedSearchableFiles.computeIfAbsent(dataFormat, k -> new ArrayList<>()).add(writerFiles);
+ }));
+ }
+
+ @Override
+ public void writeTo(StreamOutput out) throws IOException {
+ super.writeTo(out);
+
+ // Write userData map
+ if (userData == null) {
+ out.writeVInt(0);
+ } else {
+ out.writeVInt(userData.size());
+ for (Map.Entry entry : userData.entrySet()) {
+ out.writeString(entry.getKey());
+ out.writeString(entry.getValue());
+ }
+ }
+
+ out.writeLong(lastWriterGeneration);
+
+ out.writeVInt(segmentList != null ? segmentList.size() : 0);
+ if (segmentList != null) {
+ for (Segment segment : segmentList) {
+ segment.writeTo(out);
+ }
+ }
+ }
+
+ public String serializeToString() throws IOException {
+ try (BytesStreamOutput out = new BytesStreamOutput()) {
+ this.writeTo(out);
+ return Base64.getEncoder().encodeToString(out.bytes().toBytesRef().bytes);
+ }
+ }
+
+ public static CompositeEngineCatalogSnapshot deserializeFromString(String serializedData) throws IOException {
+ byte[] bytes = Base64.getDecoder().decode(serializedData);
+ try (BytesStreamInput in = new BytesStreamInput(bytes)) {
+ return new CompositeEngineCatalogSnapshot(in);
+ }
+ }
+
+ public Collection getSearchableFiles(String dataFormat) {
+ if (dfGroupedSearchableFiles.containsKey(dataFormat)) {
+ return dfGroupedSearchableFiles.get(dataFormat);
+ }
+ return Collections.emptyList();
+ }
+
+ public List getSegments() {
+ return segmentList;
+ }
+
+ public Collection getFileMetadataList() throws IOException {
+ Collection segments = getSegments();
+ Collection allFileMetadata = new ArrayList<>();
+
+ for (Segment segment : segments) {
+ segment.getDFGroupedSearchableFiles().forEach((dataFormatName, writerFileSet) -> {
+ for (String filePath : writerFileSet.getFiles()) {
+ File file = new File(filePath);
+ String fileName = file.getName();
+ FileMetadata fileMetadata = new FileMetadata(
+ dataFormatName,
+ fileName
+ );
+ allFileMetadata.add(fileMetadata);
+ }
+ });
+ }
+
+ return allFileMetadata;
+ }
+
+ /**
+ * Returns user data associated with this catalog snapshot.
+ *
+ * @return map of user data key-value pairs
+ */
+ public Map getUserData() {
+ return userData;
+ }
+
+ @Override
+ protected void closeInternal() {
+ // Notify to FileDeleter to remove references of files referenced in this CatalogSnapshot
+ indexFileDeleterSupplier.get().removeFileReferences(this);
+ // Remove entry from catalogSnapshotMap
+ catalogSnapshotMap.remove(generation);
+ }
+
+ public long getLastWriterGeneration() {
+ return lastWriterGeneration;
+ }
+
+ public Set getDataFormats() {
+ return dfGroupedSearchableFiles.keySet();
+ }
+
+ // used only when catalog snapshot is created from last commited segment and hence the object is not initialized with the deleter and map
+ public void setIndexFileDeleterSupplier(Supplier supplier) {
+ if (this.indexFileDeleterSupplier == null) {
+ this.indexFileDeleterSupplier = supplier;
+ }
+ }
+
+ @Override
+ public void setCatalogSnapshotMap(Map catalogSnapshotMap) {
+ this.catalogSnapshotMap = (Map) catalogSnapshotMap;
+ }
+
+ @Override
+ public void setUserData(Map userData, boolean b) {
+ if (userData == null) {
+ this.userData = Collections.emptyMap();
+ } else {
+ this.userData = new HashMap<>(userData);
+ }
+ }
+
+ @Override
+ public long getId() {
+ return generation;
+ }
+
+ @Override
+ public CompositeEngineCatalogSnapshot clone() {
+ CompositeEngineCatalogSnapshot cloned = new CompositeEngineCatalogSnapshot(
+ this.generation,
+ this.version,
+ new ArrayList<>(this.segmentList),
+ this.catalogSnapshotMap,
+ this.indexFileDeleterSupplier
+ );
+ cloned.userData = new HashMap<>(this.userData);
+ cloned.lastWriterGeneration = this.lastWriterGeneration;
+ return cloned;
+ }
+
+ @Override
+ public String toString() {
+ return "CatalogSnapshot{" + "id=" + generation + ", version=" + version + ", dfGroupedSearchableFiles=" + dfGroupedSearchableFiles + ", List of Segment= " + segmentList + ", userData=" + userData +'}';
+ }
+}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/IndexFileDeleter.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/IndexFileDeleter.java
index d365187b1e487..25a720cd56703 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/coord/IndexFileDeleter.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/IndexFileDeleter.java
@@ -83,8 +83,10 @@ private Map> segregateFilesByFormat(CatalogSnapshot s
Collection dfFiles = new HashSet<>();
Collection fileSets = snapshot.getSearchableFiles(dataFormat);
for (WriterFileSet fileSet : fileSets) {
+ Path directory = Path.of(fileSet.getDirectory());
for (String file : fileSet.getFiles()) {
- dfFiles.add(fileSet.getDirectory() + "/" + file);
+ // ToDo: @Shreyansh update this to relative path
+ dfFiles.add(directory.resolve(file).toAbsolutePath().normalize().toString());
}
}
dfSegregatedFiles.put(dataFormat, dfFiles);
@@ -100,15 +102,18 @@ private void deleteUnreferencedFiles(ShardPath shardPath) throws IOException {
String dataFormat = entry.getKey();
Collection referencedFiles = entry.getValue().keySet();
Collection filesToDelete = new HashSet<>();
- // TODO - Currently hardcoding to get all parquet files in data path. Fix this
- try (DirectoryStream stream = Files.newDirectoryStream(shardPath.getDataPath(), "*.parquet")) {
+ Path dataFormatPath = shardPath.getDataPath().resolve(dataFormat);
+ if (!Files.exists(dataFormatPath)) continue;
+
+ try (DirectoryStream stream = Files.newDirectoryStream(dataFormatPath, "*." + dataFormat)) {
StreamSupport.stream(stream.spliterator(), false)
- .map(Path::toString)
+ .map(p -> p.toAbsolutePath().normalize().toString())
.filter((file) -> (!referencedFiles.contains(file)))
.forEach(filesToDelete::add);
}
- filesToDelete = filesToDelete.stream().map(file -> shardPath.getDataPath().resolve(file).toString()).collect(Collectors.toSet());
- dfFilesToDelete.put(dataFormat, filesToDelete);
+ if (!filesToDelete.isEmpty()) {
+ dfFilesToDelete.put(dataFormat, filesToDelete);
+ }
}
deleteUnreferencedFiles(dfFilesToDelete);
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/Segment.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/Segment.java
new file mode 100644
index 0000000000000..48fa6645b7757
--- /dev/null
+++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/Segment.java
@@ -0,0 +1,82 @@
+/*
+ * 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.engine.exec.coord;
+
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.common.io.stream.StreamOutput;
+import org.opensearch.core.common.io.stream.Writeable;
+import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.engine.exec.WriterFileSet;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Represents a segment in the catalog snapshot containing files grouped by data format.
+ */
+public class Segment implements Serializable, Writeable {
+
+ private final long generation;
+ private final Map dfGroupedSearchableFiles;
+
+ public Segment(long generation) {
+ this.dfGroupedSearchableFiles = new HashMap<>();
+ this.generation = generation;
+ }
+
+ public Segment(StreamInput in) throws IOException {
+ this.generation = in.readLong();
+ this.dfGroupedSearchableFiles = new HashMap<>();
+ int mapSize = in.readVInt();
+ for (int i = 0; i < mapSize; i++) {
+ String dataFormat = in.readString();
+ WriterFileSet writerFileSet = new WriterFileSet(in);
+ dfGroupedSearchableFiles.put(dataFormat, writerFileSet);
+ }
+ }
+
+ public void addSearchableFiles(String dataFormat, WriterFileSet writerFileSetGroup) {
+ dfGroupedSearchableFiles.put(dataFormat, writerFileSetGroup);
+ }
+
+ public Map getDFGroupedSearchableFiles() {
+ return dfGroupedSearchableFiles;
+ }
+
+ public Collection getSearchableFiles(String df) {
+ List searchableFiles = new ArrayList<>();
+ WriterFileSet fileSet = dfGroupedSearchableFiles.get(df);
+ if (fileSet != null) {
+ String directory = fileSet.getDirectory();
+ for (String file : fileSet.getFiles()) {
+ searchableFiles.add(new FileMetadata(df, file));
+ }
+ }
+ return searchableFiles;
+ }
+
+ public long getGeneration() {
+ return generation;
+ }
+
+ @Override
+ public void writeTo(StreamOutput out) throws IOException {
+ out.writeLong(generation);
+ out.writeVInt(dfGroupedSearchableFiles.size());
+ for (Map.Entry entry : dfGroupedSearchableFiles.entrySet()) {
+ out.writeString(entry.getKey());
+ entry.getValue().writeTo(out);
+ }
+ }
+}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/coord/SegmentInfosCatalogSnapshot.java b/server/src/main/java/org/opensearch/index/engine/exec/coord/SegmentInfosCatalogSnapshot.java
index 03883a7bb001a..5521d987de952 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/coord/SegmentInfosCatalogSnapshot.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/coord/SegmentInfosCatalogSnapshot.java
@@ -16,20 +16,24 @@
import org.opensearch.core.common.io.stream.StreamInput;
import org.opensearch.core.common.io.stream.StreamOutput;
import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.engine.exec.WriterFileSet;
import java.io.IOException;
+import java.nio.file.Path;
import java.util.Collection;
import java.util.List;
import java.util.Map;
-import java.util.function.Supplier;
+import java.util.Set;
import java.util.stream.Collectors;
public class SegmentInfosCatalogSnapshot extends CatalogSnapshot {
+ private static final String CATALOG_SNAPSHOT_KEY = "_segment_infos_catalog_snapshot_";
+
private final SegmentInfos segmentInfos;
- public SegmentInfosCatalogSnapshot(long id, long version, List segmentList, Map catalogSnapshotMap, Supplier indexFileDeleterSupplier, SegmentInfos segmentInfos) {
- super(id, version, segmentList, catalogSnapshotMap, indexFileDeleterSupplier);
+ public SegmentInfosCatalogSnapshot(SegmentInfos segmentInfos) {
+ super(CATALOG_SNAPSHOT_KEY + segmentInfos.getGeneration(), segmentInfos.getGeneration(), segmentInfos.getVersion());
this.segmentInfos = segmentInfos;
}
@@ -55,10 +59,76 @@ public void writeTo(StreamOutput out) throws IOException {
@Override
public Collection getFileMetadataList() throws IOException {
- return segmentInfos.files(true).stream().map(file -> new FileMetadata(file, "lucene")).collect(Collectors.toList());
+ return segmentInfos.files(true).stream().map(file -> new FileMetadata("lucene", file)).collect(Collectors.toList());
}
public SegmentInfos getSegmentInfos() {
return segmentInfos;
}
+
+ @Override
+ public Map getUserData() {
+ return segmentInfos.getUserData();
+ }
+
+ @Override
+ public long getId() {
+ return generation;
+ }
+
+ @Override
+ public List getSegments() {
+ throw new UnsupportedOperationException("SegmentInfosCatalogSnapshot does not support getSegments()");
+ }
+
+ @Override
+ public Collection getSearchableFiles(String dataFormat) {
+ throw new UnsupportedOperationException("SegmentInfosCatalogSnapshot does not support getSearchableFiles()");
+ }
+
+ @Override
+ public Set getDataFormats() {
+ throw new UnsupportedOperationException("SegmentInfosCatalogSnapshot does not support getDataFormats()");
+ }
+
+ @Override
+ public long getLastWriterGeneration() {
+ return -1;
+ }
+
+ @Override
+ public String serializeToString() throws IOException {
+ throw new UnsupportedOperationException("SegmentInfosCatalogSnapshot does not support serializeToString()");
+ }
+
+ @Override
+ public void remapPaths(Path newShardDataPath) {
+ // No-op for SegmentInfosCatalogSnapshot
+ }
+
+ @Override
+ public void setIndexFileDeleterSupplier(java.util.function.Supplier supplier) {
+ // No-op for SegmentInfosCatalogSnapshot
+ }
+
+ @Override
+ public void setCatalogSnapshotMap(Map catalogSnapshotMap) {
+ // No-op for SegmentInfosCatalogSnapshot
+ }
+
+ @Override
+ public SegmentInfosCatalogSnapshot clone() {
+ return new SegmentInfosCatalogSnapshot(segmentInfos);
+ }
+
+ @Override
+ protected void closeInternal() {
+ // TODO no op since SegmentInfosCatalogSnapshot is not refcounted
+ }
+
+ @Override
+ public void setUserData(Map userData, boolean b)
+ {
+ // TODO no op since SegmentInfosCatalogSnapshot is not refcounted
+ }
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergeHandler.java b/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergeHandler.java
index e9aaeffebca5e..6786e041ca9ea 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergeHandler.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergeHandler.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec.merge;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.common.logging.Loggers;
@@ -50,12 +52,12 @@ public Collection findForceMerges(int maxSegmentCount) {
try (CompositeEngine.ReleasableRef catalogSnapshotReleasableRef = compositeEngine.acquireSnapshot()) {
CatalogSnapshot catalogSnapshot = catalogSnapshotReleasableRef.getRef();
- List segmentList = catalogSnapshot.getSegments();
- List> mergeCandidates =
+ List segmentList = catalogSnapshot.getSegments();
+ List> mergeCandidates =
mergePolicy.findForceMergeCandidates(segmentList, maxSegmentCount);
// Process merge candidates
- for (List mergeGroup : mergeCandidates) {
+ for (List mergeGroup : mergeCandidates) {
oneMerges.add(new OneMerge(mergeGroup));
}
} catch (Exception e) {
@@ -71,12 +73,12 @@ public Collection findMerges() {
try (CompositeEngine.ReleasableRef catalogSnapshotReleasableRef = compositeEngine.acquireSnapshot()) {
CatalogSnapshot catalogSnapshot = catalogSnapshotReleasableRef.getRef();
- List segmentList = catalogSnapshot.getSegments();
- List> mergeCandidates =
+ List segmentList = catalogSnapshot.getSegments();
+ List> mergeCandidates =
mergePolicy.findMergeCandidates(segmentList);
// Process merge candidates
- for (List mergeGroup : mergeCandidates) {
+ for (List mergeGroup : mergeCandidates) {
oneMerges.add(new OneMerge(mergeGroup));
}
} catch (Exception e) {
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergePolicy.java b/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergePolicy.java
index c6f0c88a1ab88..f53e5efa0aba0 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergePolicy.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/merge/CompositeMergePolicy.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec.merge;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.codecs.Codec;
@@ -66,8 +68,8 @@ public void close() throws IOException {
};
}
- public List> findForceMergeCandidates(List segments, int maxSegmentCount) throws IOException {
- Map segmentMap = new HashMap<>();
+ public List> findForceMergeCandidates(List segments, int maxSegmentCount) throws IOException {
+ Map segmentMap = new HashMap<>();
SegmentInfos segmentInfos = convertToSegmentInfos(segments, segmentMap);
Map segmentsToMerge = new HashMap<>();
@@ -84,8 +86,8 @@ public List> findForceMergeCandidates(List> findMergeCandidates(List segments) throws IOException {
- Map segmentMap = new HashMap<>();
+ public List> findMergeCandidates(List segments) throws IOException {
+ Map segmentMap = new HashMap<>();
SegmentInfos segmentInfos = convertToSegmentInfos(segments, segmentMap);
try {
@@ -100,12 +102,12 @@ public List> findMergeCandidates(List segments,
- Map segmentMap
+ List segments,
+ Map segmentMap
) throws IOException {
SegmentInfos segmentInfos = new SegmentInfos(Version.LATEST.major);
- for (CatalogSnapshot.Segment segment : segments) {
+ for (Segment segment : segments) {
SegmentWrapper wrapper = new SegmentWrapper(segment, calculateTotalSize(segment), calculateNumDocs(segment));
segmentInfos.add(wrapper);
segmentMap.put(wrapper, segment);
@@ -114,15 +116,15 @@ private SegmentInfos convertToSegmentInfos(
return segmentInfos;
}
- private List> convertMergeSpecification(
+ private List> convertMergeSpecification(
MergePolicy.MergeSpecification mergeSpecification,
- Map segmentMap
+ Map segmentMap
) {
- List> merges = new ArrayList<>();
+ List> merges = new ArrayList<>();
if (mergeSpecification != null) {
for (MergePolicy.OneMerge merge : mergeSpecification.merges) {
- List segmentMerge = new ArrayList<>();
+ List segmentMerge = new ArrayList<>();
for (SegmentCommitInfo segment : merge.segments) {
segmentMerge.add(segmentMap.get(segment));
}
@@ -153,7 +155,7 @@ public Set getMergingSegments() {
return Collections.unmodifiableSet(mergingSegments);
}
- private long calculateNumDocs(CatalogSnapshot.Segment segment) {
+ private long calculateNumDocs(Segment segment) {
try {
return segment.getDFGroupedSearchableFiles().values()
.stream()
@@ -166,7 +168,7 @@ private long calculateNumDocs(CatalogSnapshot.Segment segment) {
return 0;
}
- private long calculateTotalSize(CatalogSnapshot.Segment segment) {
+ private long calculateTotalSize(Segment segment) {
try {
return segment.getDFGroupedSearchableFiles().values()
.stream()
@@ -179,9 +181,9 @@ private long calculateTotalSize(CatalogSnapshot.Segment segment) {
return 0;
}
- public synchronized void addMergingSegment(Collection segments) {
+ public synchronized void addMergingSegment(Collection segments) {
try {
- for (CatalogSnapshot.Segment segment : segments) {
+ for (Segment segment : segments) {
SegmentWrapper wrapper = new SegmentWrapper(segment, calculateTotalSize(segment), calculateNumDocs(segment));
mergingSegments.add(wrapper);
}
@@ -191,11 +193,11 @@ public synchronized void addMergingSegment(Collection s
}
}
- public synchronized void removeMergingSegment(Collection segments) {
+ public synchronized void removeMergingSegment(Collection segments) {
List segmentToRemove = new ArrayList<>();
try {
- for (CatalogSnapshot.Segment segment : segments) {
+ for (Segment segment : segments) {
SegmentWrapper wrapper = new SegmentWrapper(segment, calculateTotalSize(segment), calculateNumDocs(segment));
segmentToRemove.add(wrapper);
}
@@ -209,7 +211,7 @@ public synchronized void removeMergingSegment(Collection dataFormatMergerMap;
private final Deque mergingSegments = new ArrayDeque<>();
- private final Set currentlyMergingSegments = new HashSet<>();
+ private final Set currentlyMergingSegments = new HashSet<>();
private final Logger logger;
private final ShardId shardId;
@@ -77,7 +79,7 @@ public synchronized void updatePendingMerges() {
Collection oneMerges = findMerges();
for (OneMerge oneMerge : oneMerges) {
boolean isValidMerge = true;
- for (CatalogSnapshot.Segment segment : oneMerge.getSegmentsToMerge()) {
+ for (Segment segment : oneMerge.getSegmentsToMerge()) {
if (currentlyMergingSegments.contains(segment)) {
isValidMerge = false;
break;
@@ -92,8 +94,8 @@ public synchronized void updatePendingMerges() {
public synchronized void registerMerge(OneMerge merge) {
try (CompositeEngine.ReleasableRef catalogSnapshotReleasableRef = compositeEngine.acquireSnapshot()) {
// Validate segments exist in catalog
- List catalogSegments = catalogSnapshotReleasableRef.getRef().getSegments();
- for (CatalogSnapshot.Segment mergeSegment : merge.getSegmentsToMerge()) {
+ List catalogSegments = catalogSnapshotReleasableRef.getRef().getSegments();
+ for (Segment mergeSegment : merge.getSegmentsToMerge()) {
if (!catalogSegments.contains(mergeSegment)) {
return;
}
@@ -201,7 +203,7 @@ private void cleanupStaleMergedFiles(Map mergedWriter
private List getFilesToMerge(OneMerge oneMerge, DataFormat dataFormat) {
List writerFileSets = new ArrayList<>();
- for (CatalogSnapshot.Segment segment : oneMerge.getSegmentsToMerge()) {
+ for (Segment segment : oneMerge.getSegmentsToMerge()) {
writerFileSets.add(segment.getDFGroupedSearchableFiles().get(dataFormat.name()));
}
return writerFileSets;
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/merge/OneMerge.java b/server/src/main/java/org/opensearch/index/engine/exec/merge/OneMerge.java
index d3a015573a85d..991cdb255273c 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/merge/OneMerge.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/merge/OneMerge.java
@@ -9,23 +9,25 @@
package org.opensearch.index.engine.exec.merge;
import org.opensearch.index.engine.exec.WriterFileSet;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
import java.util.Collections;
import java.util.List;
public class OneMerge {
- private final List segmentsToMerge;
+ private final List segmentsToMerge;
private final long totalSize;
private final long totalNumDocs;
- public OneMerge(List segmentsToMerge) {
+ public OneMerge(List segmentsToMerge) {
this.segmentsToMerge = Collections.unmodifiableList(segmentsToMerge);
this.totalSize = calculateTotalSizeInBytes();
this.totalNumDocs = calculateTotalNumDocs();
}
- public List getSegmentsToMerge() {
+ public List getSegmentsToMerge() {
return segmentsToMerge;
}
diff --git a/server/src/main/java/org/opensearch/index/engine/exec/text/TextEngine.java b/server/src/main/java/org/opensearch/index/engine/exec/text/TextEngine.java
index d1b320a625729..8f43091693274 100644
--- a/server/src/main/java/org/opensearch/index/engine/exec/text/TextEngine.java
+++ b/server/src/main/java/org/opensearch/index/engine/exec/text/TextEngine.java
@@ -8,6 +8,8 @@
package org.opensearch.index.engine.exec.text;
+import org.opensearch.index.engine.exec.coord.Segment;
+
import org.opensearch.index.engine.exec.DataFormat;
import org.opensearch.index.engine.exec.DocumentInput;
import org.opensearch.index.engine.exec.FileInfos;
@@ -78,7 +80,7 @@ public void deleteFiles(Map> filesToDelete) throws IO
public RefreshResult refresh(RefreshInput refreshInput) throws IOException {
openFiles.addAll(refreshInput.getWriterFiles());
RefreshResult refreshResult = new RefreshResult();
- CatalogSnapshot.Segment segment = new CatalogSnapshot.Segment(0);
+ Segment segment = new Segment(0);
openFiles.forEach(file -> segment.addSearchableFiles(DataFormat.TEXT.name(), file));
refreshResult.setRefreshedSegments(List.of(segment));
return refreshResult;
diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java
index e8a0dda5a699e..ed537f5005b72 100644
--- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java
+++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java
@@ -14,6 +14,7 @@
import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm;
import org.opensearch.index.remote.RemoteStoreEnums.PathType;
import org.opensearch.indices.RemoteStoreSettings;
+import org.opensearch.node.remotestore.RemoteStoreNodeAttribute;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.repositories.Repository;
import org.opensearch.repositories.RepositoryMissingException;
@@ -71,4 +72,19 @@ public boolean isTranslogMetadataEnabled() {
&& blobStoreRepository.blobStore().isBlobMetadataEnabled();
}
+ public boolean isRemoteStoreRepoServerSideEncryptionEnabled() {
+ BlobStoreRepository segmentRepository, translogRepository;
+ try {
+ segmentRepository = (BlobStoreRepository) repositoriesServiceSupplier.get()
+ .repository(RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(settings));
+ translogRepository = (BlobStoreRepository) repositoriesServiceSupplier.get()
+ .repository(RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo(settings));
+ } catch (RepositoryMissingException ex) {
+ throw new IllegalArgumentException("Repository should be created before creating index with remote_store enabled setting", ex);
+ }
+ return Version.V_3_3_0.compareTo(minNodeVersionSupplier.get()) <= 0
+ && remoteStoreSettings.isClusterServerSideEncryptionEnabled()
+ && segmentRepository.isSeverSideEncryptionEnabled()
+ && translogRepository.isSeverSideEncryptionEnabled();
+ }
}
diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java
index 32a1ca0e5d5ab..3b3b9b729bc39 100644
--- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java
+++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java
@@ -248,6 +248,11 @@ public static Map determineRemoteStoreCustomMetadataDuringMigrat
return remoteCustomData;
}
+ public static boolean isServerSideEncryptionEnabledIndex(IndexMetadata indexMetadata) {
+ Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY);
+ return remoteCustomData != null && "true".equalsIgnoreCase(remoteCustomData.get(IndexMetadata.REMOTE_STORE_SSE_ENABLED_INDEX_KEY));
+ }
+
/**
* Fetches segment and translog repository names from remote store node attributes.
* Returns a blank {@link HashMap} if the cluster does not contain any remote nodes.
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 24596eee8ae37..0ac0f2548ce29 100644
--- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java
+++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java
@@ -48,11 +48,7 @@
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.search.Sort;
-import org.apache.lucene.store.AlreadyClosedException;
-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.apache.lucene.store.*;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.Version;
import org.opensearch.ExceptionsHelper;
@@ -62,6 +58,7 @@
import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.opensearch.action.admin.indices.streamingingestion.state.ShardIngestionState;
import org.opensearch.action.admin.indices.upgrade.post.UpgradeRequest;
+import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.action.support.replication.PendingReplicationActions;
import org.opensearch.action.support.replication.ReplicationResponse;
import org.opensearch.cluster.metadata.DataStream;
@@ -98,6 +95,7 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.BigArrays;
+import org.opensearch.common.util.CancellableThreads;
import org.opensearch.common.util.concurrent.AbstractAsyncTask;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.common.util.concurrent.AsyncIOProcessor;
@@ -138,6 +136,7 @@
import org.opensearch.index.engine.IngestionEngine;
import org.opensearch.index.engine.MergedSegmentWarmerFactory;
import org.opensearch.index.engine.NRTReplicationEngine;
+import org.opensearch.index.engine.NRTReplicationCompositeEngine;
import org.opensearch.index.engine.ReadOnlyEngine;
import org.opensearch.index.engine.RefreshFailedEngineException;
import org.opensearch.index.engine.SafeCommitInfo;
@@ -151,6 +150,7 @@
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.engine.exec.coord.SegmentInfosCatalogSnapshot;
import org.opensearch.index.fielddata.FieldDataStats;
import org.opensearch.index.fielddata.ShardFieldData;
import org.opensearch.index.flush.FlushStats;
@@ -171,6 +171,7 @@
import org.opensearch.index.remote.RemoteSegmentStats;
import org.opensearch.index.remote.RemoteStorePathStrategy;
import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory;
+import org.opensearch.index.remote.RemoteStoreUtils;
import org.opensearch.index.search.stats.SearchStats;
import org.opensearch.index.search.stats.ShardSearchStats;
import org.opensearch.index.seqno.LocalCheckpointTracker;
@@ -183,8 +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.*;
+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.StoreFileMetadata;
+import org.opensearch.index.store.StoreStats;
+import org.opensearch.index.store.UploadedSegmentMetadata;
import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata;
import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory;
import org.opensearch.index.translog.RemoteFsTranslog;
@@ -210,6 +217,7 @@
import org.opensearch.indices.recovery.RecoverySettings;
import org.opensearch.indices.recovery.RecoveryState;
import org.opensearch.indices.recovery.RecoveryTarget;
+import org.opensearch.indices.replication.CompositeStoreDirectoryStatsWrapper;
import org.opensearch.indices.replication.checkpoint.MergedSegmentCheckpoint;
import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher;
import org.opensearch.indices.replication.checkpoint.ReferencedSegmentsCheckpoint;
@@ -260,6 +268,7 @@
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
+import static org.opensearch.action.support.PlainActionFuture.newFuture;
import static org.opensearch.index.seqno.RetentionLeaseActions.RETAIN_ALL;
import static org.opensearch.index.seqno.SequenceNumbers.LOCAL_CHECKPOINT_KEY;
import static org.opensearch.index.seqno.SequenceNumbers.MAX_SEQ_NO;
@@ -551,7 +560,7 @@ public boolean shouldCache(Query query) {
this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory;
this.recoverySettings = recoverySettings;
this.remoteStoreSettings = remoteStoreSettings;
- this.fileDownloader = new RemoteStoreFileDownloader(shardRouting.shardId(), threadPool, recoverySettings);
+ this.fileDownloader = new RemoteStoreFileDownloader(shardRouting.shardId(), threadPool, recoverySettings, isOptimizedIndex());
this.shardMigrationState = getShardMigrationState(indexSettings, seedRemote);
this.discoveryNodes = discoveryNodes;
this.segmentReplicationStatsProvider = segmentReplicationStatsProvider;
@@ -1174,7 +1183,7 @@ public Engine.IndexResult applyIndexOperationOnReplica(
Engine.Operation.Origin.REPLICA,
sourceToParse,
id,
- null
+ getIndexer()::documentInput
);
}
@@ -1533,9 +1542,14 @@ public DocsStats docStats() {
* @throws AlreadyClosedException if shard is closed
*/
public CommitStats commitStats() {
- return getStatsHolder().commitStats();
+ final StatsHolder statsHolder = getStatsHolderOrNull();
+ if (statsHolder == null) {
+ throw new AlreadyClosedException("engine is closed");
+ }
+ return statsHolder.commitStats();
}
+
/**
* @return {@link SeqNoStats}
* @throws AlreadyClosedException if shard is closed
@@ -1830,6 +1844,10 @@ public void finalizeReplication(SegmentInfos infos) throws IOException {
* @throws IOException if an error occurs during replication finalization
*/
public void finalizeReplication(CatalogSnapshot catalogSnapshot, ReplicationCheckpoint replicationCheckpoint) throws IOException {
+ if (catalogSnapshot instanceof SegmentInfosCatalogSnapshot) {
+ finalizeReplication(((SegmentInfosCatalogSnapshot) catalogSnapshot).getSegmentInfos());
+ return;
+ }
if (Thread.holdsLock(mutex)) {
throw new IllegalStateException("finalizeReplication must not be called under mutex - potential deadlock risk");
}
@@ -1931,7 +1949,7 @@ public GatedCloseable acquireSafeIndexCommit() throws EngineExcepti
final IndexShardState state = this.state; // one time volatile read
// we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine
if (state == IndexShardState.STARTED || state == IndexShardState.CLOSED) {
- return getIndexingExecutionCoordinator().acquireSafeIndexCommit();
+ return getIndexer().acquireSafeIndexCommit();
} else {
throw new IllegalIndexShardStateException(shardId, state, "snapshot is not allowed");
}
@@ -1991,6 +2009,9 @@ public Tuple, ReplicationCheckpoint> getLatestSegme
* TODO: SegRep changes for decoupling. looks to depend on codec.
*/
ReplicationCheckpoint computeReplicationCheckpoint(CatalogSnapshot catalogSnapshot) throws IOException {
+ if (catalogSnapshot instanceof SegmentInfosCatalogSnapshot) {
+ return computeReplicationCheckpoint(((SegmentInfosCatalogSnapshot) catalogSnapshot).getSegmentInfos());
+ }
if (catalogSnapshot == null) {
return ReplicationCheckpoint.empty(shardId);
}
@@ -2011,7 +2032,7 @@ ReplicationCheckpoint computeReplicationCheckpoint(CatalogSnapshot catalogSnapsh
catalogSnapshot.getVersion(),
formatAwareMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(),
formatAwareMetadataMap,
- getEngine().config().getCodec().getName()
+ getIndexer().config().getCodec().getName()
);
logger.trace("Recomputed ReplicationCheckpoint from CatalogSnapshot for shard {}", checkpoint);
return checkpoint;
@@ -2022,16 +2043,26 @@ ReplicationCheckpoint computeReplicationCheckpoint(CatalogSnapshot catalogSnapsh
* Creates a mapping from FileMetadata to StoreFileMetadata preserving format information.
*/
private Map extractFormatAwareMetadata(CatalogSnapshot catalogSnapshot) throws IOException {
+ if (!isOptimizedIndex()) {
+ return getSegmentMetadataMap().entrySet().stream().collect(
+ Collectors.toMap(
+ e -> new FileMetadata("lucene", e.getKey()),
+ Map.Entry::getValue
+ )
+ );
+ }
Map formatAwareMap = new HashMap<>();
- if(catalogSnapshot == null){
+ if (catalogSnapshot == null) {
return formatAwareMap;
}
for (FileMetadata fileMetadata : catalogSnapshot.getFileMetadataList()) {
try {
- long fileLength = store.compositeStoreDirectory().fileLength(fileMetadata);
- long checksum = store.compositeStoreDirectory().calculateChecksum(fileMetadata);
+ Directory storeDirectory = isOptimizedIndex() ? store.compositeStoreDirectory() : store().directory();
+ String fileName = isOptimizedIndex() ? fileMetadata.serialize() : fileMetadata.file();
+ long fileLength = storeDirectory.fileLength(fileName);
+ long checksum = ((CompositeStoreDirectory) storeDirectory).calculateChecksum(fileMetadata);
StoreFileMetadata storeFileMetadata = new StoreFileMetadata(
fileMetadata.file(),
@@ -2248,8 +2279,15 @@ public Store.MetadataSnapshot snapshotStoreMetadata() throws IOException {
logger.debug("CompositeEngine deletion policy not initialized during peer recovery, falling back to direct store access for shard [{}]", shardId);
wrappedIndexCommit = null;
}
+ } else {
+ // Use regular Engine for non-optimized indices
+ Engine engine = currentEngineReference.get();
+ if (engine != null) {
+ wrappedIndexCommit = engine.acquireSafeIndexCommit();
+ }
}
if (wrappedIndexCommit == null) {
+ // Only use direct store access when no engine is running
return store.getMetadata(null, true);
}
}
@@ -3021,7 +3059,17 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b
// we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata).
final Engine newEngine = engineFactory.newReadWriteEngine(config);
if (indexSettings.isOptimizedIndex()) {
- CompositeEngine compositeEngine = new CompositeEngine(
+ CompositeEngine compositeEngine = config.isReadOnlyReplica()
+ ? new NRTReplicationCompositeEngine(
+ config,
+ mapperService,
+ pluginsService,
+ indexSettings,
+ path,
+ LocalCheckpointTracker::new,
+ TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER
+ )
+ : new CompositeEngine(
config,
mapperService,
pluginsService,
@@ -3030,6 +3078,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b
LocalCheckpointTracker::new,
TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER
);
+ // Don't set currentCompositeEngineReference for replicas
currentCompositeEngineReference.set(compositeEngine);
}
onNewEngine(newEngine);
@@ -3068,6 +3117,7 @@ private boolean assertSequenceNumbersInCommit() throws IOException {
+ "] is different than engine ["
+ getHistoryUUID()
+ "]";
+
assert userData.containsKey(Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID) : "opening index which was created post 5.5.0 but "
+ Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID
+ " is not found in commit";
@@ -3247,9 +3297,14 @@ public long getIndexBufferRAMBytesUsed() {
}
public long getNativeBytesUsed() {
- return getIndexer().getNativeBytesUsed();
+ Indexer indexer = getIndexer();
+ if (indexer == null) {
+ return 0;
+ }
+ return indexer.getNativeBytesUsed();
}
+
public void addShardFailureCallback(Consumer onShardFailure) {
this.shardEventListener.delegates.add(onShardFailure);
}
@@ -4157,7 +4212,7 @@ public CheckpointState getCheckpointState() {
}
public StatsHolder getStatsHolder() {
- return indexSettings.isOptimizedIndex() ? getIndexingExecutionCoordinator(): currentEngineReference.get();
+ return indexSettings.isOptimizedIndex() ? getIndexingExecutionCoordinator() : currentEngineReference.get();
}
public IndexingThrottler getIndexingThrottler() {
@@ -4529,6 +4584,10 @@ public boolean isRemoteTranslogEnabled() {
return indexSettings() != null && (indexSettings().isRemoteTranslogStoreEnabled());
}
+ public boolean isOptimizedIndex() {
+ return indexSettings().isOptimizedIndex();
+ }
+
/**
* This checks if we are in state to upload to remote store. Until the cluster-manager informs the shard through
* cluster state, the shard will not be in STARTED state. This method is used to prevent pre-emptive segment or
@@ -5340,13 +5399,29 @@ public void afterRefresh(boolean didRefresh) throws IOException {
}
private void updateReplicationCheckpoint() {
- try (CompositeEngine.ReleasableRef catalogSnapshotRef = getCatalogSnapshotFromEngine()) {
- final ReplicationCheckpoint checkpoint = computeReplicationCheckpoint(catalogSnapshotRef.getRef());
- replicationTracker.setLatestReplicationCheckpoint(checkpoint);
- logger.trace("Updated replication checkpoint from CatalogSnapshot: shard={}, checkpoint={}", shardId, checkpoint);
- } catch (Exception 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);
+ if (isOptimizedIndex()) {
+ CompositeEngine compositeEngine = currentCompositeEngineReference.get();
+ // Use CompositeEngine's CatalogSnapshot for optimized indices
+ try (CompositeEngine.ReleasableRef catalogSnapshotRef = compositeEngine.acquireSnapshot()) {
+ final ReplicationCheckpoint checkpoint = computeReplicationCheckpoint(catalogSnapshotRef.getRef());
+ replicationTracker.setLatestReplicationCheckpoint(checkpoint);
+ } catch (Exception e) {
+ logger.error("Error computing replication checkpoint from catalog snapshot for shard [{}]", shardId, e);
+ }
+ } else {
+ // Fall back to standard engine for non-optimized segment replication
+ Engine engine = getEngineOrNull();
+ if (engine == null) {
+ logger.debug("Skipping replication checkpoint update - engine not initialized yet for shard [{}]", shardId);
+ return;
+ }
+ try (GatedCloseable segmentInfosSnapshot = engine.getSegmentInfosSnapshot()) {
+ final ReplicationCheckpoint checkpoint = computeReplicationCheckpoint(segmentInfosSnapshot.get());
+ replicationTracker.setLatestReplicationCheckpoint(checkpoint);
+ logger.trace("Updated replication checkpoint from SegmentInfos: shard={}, checkpoint={}", shardId, checkpoint);
+ } catch (Exception e) {
+ logger.error("Error computing replication checkpoint from engine for shard [{}]", shardId, e);
+ }
}
}
@@ -5447,17 +5522,7 @@ public void close() throws IOException {
if ((indexSettings.isRemoteTranslogStoreEnabled() || this.isRemoteSeeded()) && shardRouting.primary()) {
syncRemoteTranslogAndUpdateGlobalCheckpoint();
}
- newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker)));
- onNewEngine(newEngineReference.get());
- }
- final TranslogRecoveryRunner translogRunner = (snapshot) -> runTranslogRecovery(
- newEngineReference.get(),
- snapshot,
- Engine.Operation.Origin.LOCAL_RESET,
- () -> {
- // TODO: add a dedicate recovery stats for the reset translog
- }
- );
+ }
// When the new engine is created, translogs are synced from remote store onto local. Since remote store is the source
// of truth for translog, we play all translogs that exists locally. Otherwise, the recoverUpto happens upto global checkpoint.
@@ -5466,13 +5531,66 @@ public void close() throws IOException {
long recoverUpto = this.isRemoteTranslogEnabled() || indexSettings().isSegRepEnabledOrRemoteNode()
? Long.MAX_VALUE
: globalCheckpoint;
- newEngineReference.get()
- .translogManager()
- .recoverFromTranslog(translogRunner, newEngineReference.get().getProcessedLocalCheckpoint(), recoverUpto);
- newEngineReference.get().refresh("reset_engine");
+
+ // Only create CompositeEngine for optimized indices
+ CompositeEngine newCompositeEngine;
+ if (indexSettings.isOptimizedIndex()) {
+ // Create NEW CompositeEngine OUTSIDE synchronized block with fresh translog
+ newCompositeEngine = new CompositeEngine(
+ newEngineConfig(replicationTracker),
+ mapperService,
+ pluginsService,
+ indexSettings,
+ path,
+ LocalCheckpointTracker::new,
+ TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER
+ );
+
+ final TranslogRecoveryRunner translogRunner = (snapshot) -> runTranslogRecovery(
+ newCompositeEngine,
+ snapshot,
+ Engine.Operation.Origin.LOCAL_RESET,
+ () -> {
+ // TODO: add a dedicate recovery stats for the reset translog
+ }
+ );
+
+ // Recover the NEW CompositeEngine's translog FIRST
+ newCompositeEngine
+ .translogManager()
+ .recoverFromTranslog(translogRunner, newCompositeEngine.getProcessedLocalCheckpoint(), recoverUpto);
+ newCompositeEngine.refresh("reset_engine");
+ } else {
+ newCompositeEngine = null;
+ }
+
+ // Create InternalEngine AFTER translog recovery so it reads the updated commit with correct checkpoints
+ final Engine newEngine = engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker));
+ newEngineReference.set(newEngine);
+
+ if (!indexSettings.isOptimizedIndex()) {
+ synchronized (engineMutex) {
+ onNewEngine(newEngineReference.get());
+ }
+ final TranslogRecoveryRunner translogRunner = (snapshot) -> runTranslogRecovery(
+ newEngineReference.get(),
+ snapshot,
+ Engine.Operation.Origin.LOCAL_RESET,
+ () -> {
+ // TODO: add a dedicate recovery stats for the reset translog
+ }
+ );
+ newEngineReference.get()
+ .translogManager()
+ .recoverFromTranslog(translogRunner, newEngineReference.get().getProcessedLocalCheckpoint(), recoverUpto);
+ newEngineReference.get().refresh("reset_engine");
+ }
+
synchronized (engineMutex) {
verifyNotClosed();
- IOUtils.close(currentEngineReference.getAndSet(newEngineReference.get()));
+ IOUtils.close(currentEngineReference.getAndSet(newEngineReference.get()), currentCompositeEngineReference.getAndSet(newCompositeEngine));
+
+ // onNewEngine must be called inside synchronized(engineMutex) block for both optimized and non-optimized indices
// We set active because we are now writing operations to the engine; this way,
// if we go idle after some time and become inactive, we still give sync'd flush a chance to run.
active.set(true);
@@ -5497,7 +5615,8 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException {
getThreadPool(),
indexSettings.getRemoteStorePathStrategy(),
remoteStoreSettings,
- indexSettings().isTranslogMetadataEnabled()
+ indexSettings().isTranslogMetadataEnabled(),
+ RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata())
);
}
@@ -5520,7 +5639,8 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException {
shardId,
indexSettings.getRemoteStorePathStrategy(),
indexSettings().isTranslogMetadataEnabled(),
- 0
+ 0,
+ RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata())
);
}
@@ -5530,6 +5650,24 @@ public void syncTranslogFilesFromGivenRemoteTranslog(
RemoteStorePathStrategy remoteStorePathStrategy,
boolean isTranslogMetadataEnabled,
long timestamp
+ ) throws IOException {
+ this.syncTranslogFilesFromGivenRemoteTranslog(
+ repository,
+ shardId,
+ remoteStorePathStrategy,
+ isTranslogMetadataEnabled,
+ timestamp,
+ false
+ );
+ }
+
+ public void syncTranslogFilesFromGivenRemoteTranslog(
+ Repository repository,
+ ShardId shardId,
+ RemoteStorePathStrategy remoteStorePathStrategy,
+ boolean isTranslogMetadataEnabled,
+ long timestamp,
+ boolean isServerSideEncryptionEnabled
) throws IOException {
RemoteFsTranslog.download(
repository,
@@ -5541,7 +5679,8 @@ public void syncTranslogFilesFromGivenRemoteTranslog(
logger,
shouldSeedRemoteStore(),
isTranslogMetadataEnabled,
- timestamp
+ timestamp,
+ isServerSideEncryptionEnabled
);
}
@@ -5570,36 +5709,51 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn
// are uploaded to the remote segment store.
RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init();
- Map uploadedSegments = remoteDirectory
- .getSegmentsUploadedToRemoteStore()
- .entrySet()
- .stream()
- .filter(entry -> entry.getKey().startsWith(IndexFileNames.SEGMENTS) == false)
- .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+ Map uploadedSegments = remoteDirectory.getSegmentsUploadedToRemoteStore();
+ Map filteredSegments = new HashMap<>();
+ for (Map.Entry entry : uploadedSegments.entrySet()) {
+ if (!entry.getKey().startsWith(IndexFileNames.SEGMENTS)) {
+ filteredSegments.put(entry.getKey(), entry.getValue());
+ }
+ }
store.incRef();
remoteStore.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());
+ Store.StoreDirectory directory = isOptimizedIndex() ? store().compositeStoreDirectory() : (Store.StoreDirectory) store().directory();
+ storeDirectory = new StoreRecovery.StatsDirectoryWrapper(directory, recoveryState.getIndex());
+ for (String file : filteredSegments.keySet()) {
+ long checksum = Long.parseLong(filteredSegments.get(file).getChecksum());
+ boolean fileExistsLocally;
+
+ // Parse FileMetadata from serialized key to get actual filename
FileMetadata fileMetadata = new FileMetadata(file);
- if (overrideLocal || localDirectoryContains(storeDirectory, fileMetadata, checksum) == false) {
- recoveryState.getIndex().addFileDetail(fileMetadata.file(), uploadedSegments.get(file).getLength(), false);
+ if (isOptimizedIndex() && directory instanceof CompositeStoreDirectory) {
+ fileExistsLocally = localDirectoryContains((CompositeStoreDirectory) directory, fileMetadata, checksum);
+ } else {
+ // For non-optimized indices, use the actual filename from FileMetadata
+ fileExistsLocally = localDirectoryContainsFile(storeDirectory, fileMetadata.file(), checksum);
+ }
+
+ if (overrideLocal || !fileExistsLocally) {
+ recoveryState.getIndex().addFileDetail(file, filteredSegments.get(file).getLength(), false);
} else {
- recoveryState.getIndex().addFileDetail(fileMetadata.file(), uploadedSegments.get(file).getLength(), true);
+ recoveryState.getIndex().addFileDetail(file, filteredSegments.get(file).getLength(), true);
}
}
} else {
- storeDirectory = store.directory();
+ storeDirectory = isOptimizedIndex()
+ ? store().compositeStoreDirectory()
+ : store.directory();
}
if (indexSettings.isWarmIndex() == false) {
- // ToDo:@Kamal update while restore implementation
- // copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal, onFileSync);
+ copySegmentFiles(storeDirectory, remoteDirectory, null, filteredSegments, overrideLocal, onFileSync);
}
if (remoteSegmentMetadata != null) {
+ // Remote store always stores Lucene SegmentInfos format (for both optimized and non-optimized indices)
+ // For optimized indices, the CatalogSnapshot is embedded within userData of the SegmentInfos
final SegmentInfos infosSnapshot = store.buildSegmentInfos(
remoteSegmentMetadata.getSegmentInfosBytes(),
remoteSegmentMetadata.getGeneration()
@@ -5631,7 +5785,6 @@ 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
@@ -5643,47 +5796,158 @@ public void syncSegmentsFromGivenRemoteSegmentStore(
RemoteSegmentMetadata remoteSegmentMetadata,
boolean pinnedTimestamp
) throws IOException {
- throw new UnsupportedOperationException("Not implemented yet");
+ 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) {
+ // Fix: Add isOptimizedIndex() check for optimized indices
+ Store.StoreDirectory directory = isOptimizedIndex()
+ ? store().compositeStoreDirectory()
+ : (Store.StoreDirectory) store().directory();
+ storeDirectory = new StoreRecovery.StatsDirectoryWrapper(directory, recoveryState.getIndex());
+ for (String file : uploadedSegments.keySet()) {
+ long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum());
+ boolean fileExistsLocally;
+
+ // Fix: Use format-aware checksum for optimized indices
+ FileMetadata fileMetadata = new FileMetadata(file);
+ if (isOptimizedIndex() && directory instanceof CompositeStoreDirectory) {
+ fileExistsLocally = localDirectoryContains((CompositeStoreDirectory) directory, fileMetadata, checksum);
+ } else {
+ fileExistsLocally = localDirectoryContainsFile(storeDirectory, fileMetadata.file(), checksum);
+ }
+
+ if (overrideLocal || !fileExistsLocally) {
+ recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), false);
+ } else {
+ recoveryState.getIndex().addFileDetail(file, uploadedSegments.get(file).getLength(), true);
+ }
+ }
+ } else {
+ storeDirectory = isOptimizedIndex()
+ ? store().compositeStoreDirectory()
+ : 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();
+ }
+ }
}
- // ToDo: Needs to be updated while Replication flow implementation
+ /**
+ * Unified method to copy segment files from remote store.
+ * Handles both optimized (multiformat) and non-optimized (plain Lucene) indices.
+ * For optimized indices, keys in uploadedSegments are serialized FileMetadata strings like "segment_1.si:::lucene".
+ * For non-optimized indices, keys are plain filenames like "segment_1.si".
+ */
private String copySegmentFiles(
- CompositeStoreDirectory storeDirectory,
+ Directory storeDirectory,
RemoteSegmentStoreDirectory sourceRemoteDirectory,
RemoteSegmentStoreDirectory targetRemoteDirectory,
- Map uploadedSegments,
+ Map uploadedSegments,
boolean overrideLocal,
final Runnable onFileSync
- ) throws IOException {
+ ) throws IOException {
Set toDownloadSegments = new HashSet<>();
Set skippedSegments = new HashSet<>();
String segmentNFile = null;
try {
if (overrideLocal) {
- for (FileMetadata file : storeDirectory.listFileMetadata()) {
+ for (String file : storeDirectory.listAll()) {
storeDirectory.deleteFile(file);
}
}
- for (FileMetadata file : uploadedSegments.keySet()) {
- long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum());
- if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) {
- toDownloadSegments.add(file.file());
- } else {
- skippedSegments.add(file.file());
- }
+ for (String file : uploadedSegments.keySet()) {
+ long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum());
+ boolean fileExistsLocally;
+
+ // For optimized indices with multiformat support (e.g., Parquet files),
+ // use format-aware checksum validation since Parquet files don't have Lucene codec footers
+ FileMetadata fileMetadata = new FileMetadata(file);
+ if (isOptimizedIndex() && storeDirectory instanceof CompositeStoreDirectory) {
+ fileExistsLocally = localDirectoryContains((CompositeStoreDirectory) storeDirectory, fileMetadata, checksum);
+ } else if (storeDirectory instanceof StoreRecovery.StatsDirectoryWrapper
+ && ((StoreRecovery.StatsDirectoryWrapper) storeDirectory).getDelegate() instanceof CompositeStoreDirectory) {
+ // Handle case where storeDirectory is wrapped in StatsDirectoryWrapper
+ fileExistsLocally = localDirectoryContains(
+ (CompositeStoreDirectory) ((StoreRecovery.StatsDirectoryWrapper) storeDirectory).getDelegate(),
+ fileMetadata, checksum);
+ } else {
+ fileExistsLocally = localDirectoryContainsFile(storeDirectory, fileMetadata.file(), checksum);
+ }
- if (file.file().startsWith(IndexFileNames.SEGMENTS)) {
+ if (overrideLocal || !fileExistsLocally) {
+ toDownloadSegments.add(file);
+ } else {
+ skippedSegments.add(file);
+ }
+
+ if (file.startsWith(IndexFileNames.SEGMENTS)) {
assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file";
- segmentNFile = file.file();
+ segmentNFile = file;
}
}
if (toDownloadSegments.isEmpty() == false) {
try {
- // ToDo: @Kamal, Implement while restore flow implementation.
- // fileDownloader.download(sourceRemoteDirectory, storeDirectory, targetRemoteDirectory, toDownloadSegments, onFileSync);
+ fileDownloader.download(sourceRemoteDirectory, storeDirectory, targetRemoteDirectory, toDownloadSegments, onFileSync);
} catch (Exception e) {
throw new IOException("Error occurred when downloading segments from remote store", e);
}
@@ -5696,40 +5960,72 @@ private String copySegmentFiles(
return segmentNFile;
}
- // ToDo: @Kamal
boolean localDirectoryContains(CompositeStoreDirectory localDirectory, FileMetadata fileMetadata, long checksum) throws IOException {
- throw new UnsupportedOperationException("Not implemented yet");
- }
+ try {
+ // Use existing CompositeStoreDirectory checksum calculation (format-aware)
+ long localChecksum = localDirectory.calculateChecksum(fileMetadata);
- // 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)) {
+ if (checksum == localChecksum) {
return true;
} else {
- logger.warn("Checksum mismatch between local and remote segment file: {}, will override local file", fileMetadata);
+ logger.warn("Checksum mismatch for file: {}, format: {}, expected: {}, local: {}, will override",
+ fileMetadata.file(), fileMetadata.dataFormat(), checksum, localChecksum);
// 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(fileMetadata.file());
+ localDirectory.deleteFile(fileMetadata);
} else {
// segment conflict with remote store while the shard is serving reads.
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", fileMetadata.file());
+ logger.debug("File {} with format {} does not exist in local FS, downloading from remote store",
+ fileMetadata.file(), fileMetadata.dataFormat());
} catch (IOException e) {
- 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(fileMetadata.file());
+ // Check if root cause is "file not found" - MultiFormatStoreException wraps the original exception
+ Throwable cause = e.getCause();
+ if (cause instanceof NoSuchFileException || cause instanceof FileNotFoundException) {
+ logger.debug("File {} with format {} does not exist in local FS (wrapped exception), downloading from remote store",
+ fileMetadata.file(), fileMetadata.dataFormat());
+ } else {
+ logger.warn("Exception while reading checksum of file: {}, format: {}, this can happen if file is corrupted",
+ fileMetadata.file(), fileMetadata.dataFormat(), e);
+ // For any other exception on reading checksum, we delete the file to re-download again
+ try {
+ localDirectory.deleteFile(fileMetadata);
+ } catch (NoSuchFileException | FileNotFoundException ignored) {
+ // File already doesn't exist, nothing to delete
+ }
+ }
}
return false;
}
+ boolean localDirectoryContainsFile(Directory localDirectory, String fileName, long checksum) throws IOException {
+ try (IndexInput indexInput = localDirectory.openInput(fileName, IOContext.READONCE)) {
+ if (checksum == CodecUtil.retrieveChecksum(indexInput)) {
+ return true;
+ } else {
+ logger.warn("Checksum mismatch between local and remote segment file: {}, will override local file", fileName);
+ if (isReadAllowed() == false) {
+ localDirectory.deleteFile(fileName);
+ } else {
+ failShard("Local copy of segment " + fileName + " 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", fileName);
+ } catch (IOException e) {
+ logger.warn("Exception while reading checksum of file: {}, this can happen if file is corrupted", fileName, e);
+ localDirectory.deleteFile(fileName);
+ }
+ return false;
+ }
+
+
/**
* Returns the maximum sequence number of either update or delete operations have been processed in this shard
@@ -5740,7 +6036,7 @@ boolean localDirectoryContains(Directory localDirectory, FileMetadata fileMetada
* executing that replication request on a replica.
*/
public long getMaxSeqNoOfUpdatesOrDeletes() {
- return getEngine().getMaxSeqNoOfUpdatesOrDeletes();
+ return getIndexer().getMaxSeqNoOfUpdatesOrDeletes();
}
/**
@@ -5788,7 +6084,7 @@ public GatedCloseable getSegmentInfosSnapshot() {
public CompositeEngine.ReleasableRef getCatalogSnapshotFromEngine() {
try {
- return getIndexingExecutionCoordinator().acquireSnapshot();
+ return getIndexer().acquireSnapshot();
} catch (Exception e) {
throw new OpenSearchException("Error occurred while getting catalog snapshot", e);
}
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 1628e2cfc567c..ad95c7b792259 100644
--- a/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java
+++ b/server/src/main/java/org/opensearch/index/shard/ReleasableRetryableRefreshListener.java
@@ -73,7 +73,6 @@ 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 f9f9c83a16839..dddbd059eb712 100644
--- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java
+++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java
@@ -9,7 +9,11 @@
package org.opensearch.index.shard;
import org.apache.logging.log4j.Logger;
+import org.apache.lucene.codecs.CodecUtil;
+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;
@@ -19,6 +23,7 @@
import org.opensearch.core.action.ActionListener;
import org.opensearch.index.engine.InternalEngine;
import org.opensearch.index.engine.exec.FileMetadata;
+import org.opensearch.index.engine.exec.bridge.Indexer;
import org.opensearch.index.engine.exec.coord.CatalogSnapshot;
import org.opensearch.index.engine.exec.coord.CompositeEngine;
import org.opensearch.index.remote.RemoteSegmentTransferTracker;
@@ -79,7 +84,7 @@ public final class RemoteStoreRefreshListener extends ReleasableRetryableRefresh
public static final Set EXCLUDE_FILES = Set.of("write.lock");
private final IndexShard indexShard;
- private final CompositeStoreDirectory compositeStoreDirectory;
+ private final Directory storeDirectory;
private final RemoteSegmentStoreDirectory remoteDirectory;
private final RemoteSegmentTransferTracker segmentTracker;
private final Map localSegmentChecksumMap;
@@ -98,10 +103,10 @@ public RemoteStoreRefreshListener(
super(indexShard.getThreadPool());
logger = Loggers.getLogger(getClass(), indexShard.shardId());
this.indexShard = indexShard;
- this.compositeStoreDirectory = indexShard.store().compositeStoreDirectory();
+ this.storeDirectory = indexShard.isOptimizedIndex() ? indexShard.store().compositeStoreDirectory() : indexShard.store().directory();
this.remoteDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory())
.getDelegate()).getDelegate();
- remoteStoreUploader = new RemoteStoreUploaderService(indexShard, compositeStoreDirectory, remoteDirectory);
+ remoteStoreUploader = new RemoteStoreUploaderService(indexShard, storeDirectory, this.remoteDirectory, indexShard.isOptimizedIndex());
localSegmentChecksumMap = new HashMap<>();
RemoteSegmentMetadata remoteSegmentMetadata = null;
if (indexShard.routingEntry().primary()) {
@@ -240,6 +245,7 @@ private boolean syncSegments() {
CompositeEngine.ReleasableRef catalogSnapshotRef = indexShard.getCatalogSnapshotFromEngine();
CatalogSnapshot catalogSnapshot = catalogSnapshotRef.getRef();
+
final ReplicationCheckpoint checkpoint = indexShard.computeReplicationCheckpoint(catalogSnapshot);
if (checkpoint.getPrimaryTerm() != indexShard.getOperationPrimaryTerm()) {
throw new IllegalStateException(
@@ -260,13 +266,10 @@ private boolean syncSegments() {
// Log format-aware statistics
Map formatCounts = localFilesPostRefresh.stream()
.collect(Collectors.groupingBy(
- fm -> fm.dataFormat(),
+ FileMetadata::dataFormat,
Collectors.counting()
));
- logger.debug("Format-aware segment upload initiated: totalFiles={}, formatBreakdown={}",
- localFilesPostRefresh.size(), formatCounts);
-
Map fileMetadataToSizeMap = updateLocalSizeMapAndTracker(localFilesPostRefresh);
CountDownLatch latch = new CountDownLatch(1);
@@ -275,10 +278,8 @@ private boolean syncSegments() {
@Override
public void onResponse(Void unused) {
try {
- logger.debug("New segments upload successful");
// Start metadata file upload
uploadMetadata(localFilesPostRefresh, catalogSnapshot, checkpoint);
- logger.debug("Metadata upload successful");
clearStaleFilesFromLocalSegmentChecksumMap(localFilesPostRefresh);
onSuccessfulSegmentsSync(
refreshTimeMs,
@@ -424,8 +425,10 @@ private void onSuccessfulSegmentsSync(
updateRemoteRefreshTimeAndSeqNo(refreshTimeMs, refreshClockTimeMs, refreshSeqNo);
// Reset the backoffDelayIterator for the future failures
resetBackOffDelayIterator();
- // Set the minimum sequence number for keeping translog
- indexShard.getIndexer().translogManager().setMinSeqNoToKeep(lastRefreshedCheckpoint + 1);
+ Indexer indexer = indexShard.getIndexer();
+ if (indexer != null) {
+ indexer.translogManager().setMinSeqNoToKeep(lastRefreshedCheckpoint + 1);
+ }
// Publishing the new checkpoint which is used for remote store + segrep indexes
checkpointPublisher.publish(indexShard, checkpoint);
logger.debug("onSuccessfulSegmentsSync lastRefreshedCheckpoint={} checkpoint={}", lastRefreshedCheckpoint, checkpoint);
@@ -469,22 +472,19 @@ private boolean isRefreshAfterCommitSafe() {
return false;
}
- // ToDo:@Kamal Update MaxSeqNo
void uploadMetadata(Collection