From 891477abfa2ad3f0b862eed4e9fdd9a805154eed Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Fri, 10 Oct 2025 12:03:49 +0530 Subject: [PATCH 01/10] Add changes for ServerSideEncryption support for RemoteStore Signed-off-by: Pranit Kumar --- .../repositories/s3/S3BlobStore.java | 4 +- .../repositories/s3/S3Repository.java | 56 ++++++++++++ .../TransportRemoteStoreMetadataAction.java | 8 +- .../cluster/metadata/IndexMetadata.java | 33 +++++++ .../metadata/MetadataCreateIndexService.java | 6 ++ .../common/settings/IndexScopedSettings.java | 1 + .../org/opensearch/index/IndexService.java | 5 +- .../org/opensearch/index/IndexSettings.java | 19 ++++ .../opensearch/index/shard/IndexShard.java | 12 ++- .../opensearch/index/shard/StoreRecovery.java | 11 ++- .../RemoteSegmentStoreDirectoryFactory.java | 18 ++-- .../RemoteStoreLockManagerFactory.java | 3 +- ...emoteBlobStoreInternalTranslogFactory.java | 13 ++- .../RemoteFsTimestampAwareTranslog.java | 7 +- .../index/translog/RemoteFsTranslog.java | 28 ++++-- .../opensearch/indices/IndicesService.java | 6 +- .../remotestore/RemoteStoreNodeAttribute.java | 14 +++ .../blobstore/BlobStoreProvider.java | 87 +++++++++++++++++++ .../blobstore/BlobStoreProviderFactory.java | 48 ++++++++++ .../blobstore/BlobStoreRepository.java | 72 ++++++++------- .../RemoteStoreBlobStoreProvider.java | 70 +++++++++++++++ .../opensearch/index/IndexModuleTests.java | 3 +- .../index/translog/RemoteFsTranslogTests.java | 1 + .../index/shard/IndexShardTestCase.java | 3 +- 24 files changed, 461 insertions(+), 67 deletions(-) create mode 100644 server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java create mode 100644 server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java create mode 100644 server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java 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..6d543a270785f 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 @@ -507,6 +507,62 @@ protected S3BlobStore createBlobStore() { ); } + @Override + protected S3BlobStore createClientSideEncryptedBlobStore() { + serverSideEncryptionType = ServerSideEncryption.AES256.toString(); + return new S3BlobStore( + service, + s3AsyncService, + multipartUploadEnabled, + bucket, + bufferSize, + cannedACL, + storageClass, + bulkDeletesSize, + metadata, + asyncUploadUtils, + urgentExecutorBuilder, + priorityExecutorBuilder, + normalExecutorBuilder, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher, + serverSideEncryptionType, + null, + false, + null, + "" + ); + } + + @Override + protected S3BlobStore createServerSideEncryptedBlobStore() { + serverSideEncryptionType = ServerSideEncryption.AWS_KMS.toString(); + return new S3BlobStore( + service, + s3AsyncService, + multipartUploadEnabled, + bucket, + bufferSize, + cannedACL, + storageClass, + bulkDeletesSize, + metadata, + asyncUploadUtils, + urgentExecutorBuilder, + priorityExecutorBuilder, + normalExecutorBuilder, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher, + serverSideEncryptionType, + serverSideEncryptionKmsKey, + serverSideEncryptionBucketKey, + serverSideEncryptionEncryptionContext, + expectedBucketOwner + ); + } + // only use for testing (S3RepositoryTests) @Override protected BlobStore getBlobStore() { 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..c57308b4bcca2 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 @@ -109,6 +109,7 @@ protected void doExecute(Task task, RemoteStoreMetadataRequest request, ActionLi Index index = indexMetadata.getIndex(); IndexSettings indexSettings = new IndexSettings(indexMetadata, clusterService.getSettings()); + int[] shardIds = request.shards().length == 0 ? java.util.stream.IntStream.range(0, indexMetadata.getNumberOfShards()).toArray() : Arrays.stream(request.shards()).mapToInt(Integer::parseInt).toArray(); @@ -198,7 +199,9 @@ private Map> getSegmentMetadata( IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.get(indexMetadata.getSettings()), index.getUUID(), shardId, - indexSettings.getRemoteStorePathStrategy() + indexSettings.getRemoteStorePathStrategy(), + null, + indexSettings.isRemoteStoreSSEnabled() ); Map segmentMetadataMapWithFilenames = remoteDirectory.readLatestNMetadataFiles(5); @@ -257,7 +260,8 @@ private Map> getTranslogMetadataFiles( tracker, indexSettings.getRemoteStorePathStrategy(), new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()), - RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata) + RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata), + indexSettings.isRemoteStoreSSEnabled() ); 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..9b8e6c83fa272 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -371,6 +371,7 @@ public Iterator> settings() { ); public static final String SETTING_REMOTE_STORE_ENABLED = "index.remote_store.enabled"; + public static final String SETTING_REMOTE_STORE_SSE_ENABLED = "index.remote_store.sse.enabled"; public static final String SETTING_INDEX_APPEND_ONLY_ENABLED = "index.append_only.enabled"; public static final String SETTING_REMOTE_SEGMENT_STORE_REPOSITORY = "index.remote_store.segment.repository"; @@ -414,6 +415,38 @@ public Iterator> settings() { Property.Dynamic ); + /** + * Used to specify if the index data should be persisted in the remote store. + */ + public static final Setting INDEX_REMOTE_STORE_SSE_ENABLED_SETTING = Setting.boolSetting( + SETTING_REMOTE_STORE_SSE_ENABLED, + false, + new Setting.Validator<>() { + + @Override + public void validate(final Boolean value) {} + + @Override + public void validate(final Boolean value, final Map, Object> settings) { + final Boolean isRemoteStoreEnabled = (Boolean) settings.get(INDEX_REMOTE_STORE_ENABLED_SETTING); + if (!isRemoteStoreEnabled && value) { + throw new IllegalArgumentException( + "Server Side Encryption can be enabled when " + INDEX_REMOTE_STORE_ENABLED_SETTING.getKey() + " is enabled. " + ); + } + } + + @Override + public Iterator> settings() { + final List> settings = List.of(INDEX_REMOTE_STORE_ENABLED_SETTING); + return settings.iterator(); + } + }, + Property.IndexScope, + Property.PrivateIndex, + Property.Dynamic + ); + /** * Used to specify if the index data should be persisted in the remote store. */ 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..09ecb3af613de 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1177,6 +1177,12 @@ public static void updateRemoteStoreSettings( .findFirst(); if (remoteNode.isPresent()) { + + if (RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes()) + && indexName.startsWith("sse-idx-1")) { + settingsBuilder.put(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, true); + } + translogRepo = RemoteStoreNodeAttribute.getTranslogRepoName(remoteNode.get().getAttributes()); segmentRepo = RemoteStoreNodeAttribute.getSegmentRepoName(remoteNode.get().getAttributes()); if (segmentRepo != null) { diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 7bbcdec25ce12..f595edc3bb93e 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -242,6 +242,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { // Settings for remote store enablement IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, + IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING, diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 6612ea732942d..762f30c2dad70 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -697,11 +697,12 @@ public synchronized IndexShard createShard( } remoteDirectory = ((RemoteSegmentStoreDirectoryFactory) remoteDirectoryFactory).newDirectory( - RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getNodeSettings()), + RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getSettings()), this.indexSettings.getUUID(), shardId, this.indexSettings.getRemoteStorePathStrategy(), - this.indexSettings.getRemoteStoreSegmentPathPrefix() + this.indexSettings.getRemoteStoreSegmentPathPrefix(), + this.indexSettings.isRemoteStoreEnabled() ); } // When an instance of Store is created, a shardlock is created which is released on closing the instance of store. diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index bf3fef3d83caf..8744848695f9f 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -950,6 +950,8 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { */ private final boolean isCompositeIndex; + private boolean isRemoteStoreSSEnabled; + /** * Denotes whether search via star tree index is enabled for this index */ @@ -1035,6 +1037,9 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null); replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); + isRemoteStoreSSEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false); + + System.out.println("[PRANIT]: IndexSettings.IndexSettings is for index " + this.index.getName() + " SSE ENABLED VALUE is " + isRemoteStoreSSEnabled); isWarmIndex = settings.getAsBoolean(IndexModule.IS_WARM_INDEX_SETTING.getKey(), false); @@ -1243,6 +1248,9 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti ); scopedSettings.addSettingsUpdateConsumer(ALLOW_DERIVED_FIELDS, this::setAllowDerivedField); scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, this::setRemoteStoreEnabled); + + scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, this::setRemoteStoreSseEnabled); + scopedSettings.addSettingsUpdateConsumer( IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, this::setRemoteStoreRepository @@ -1431,6 +1439,13 @@ public boolean isRemoteStoreEnabled() { return isRemoteStoreEnabled; } + /** + * Returns if remote store SSE is enabled for this index. + */ + public boolean isRemoteStoreSSEnabled() { + return isRemoteStoreSSEnabled; + } + public boolean isAssignedOnRemoteNode() { return assignedOnRemoteNode; } @@ -2141,6 +2156,10 @@ public void setRemoteStoreEnabled(boolean isRemoteStoreEnabled) { this.isRemoteStoreEnabled = isRemoteStoreEnabled; } + public void setRemoteStoreSseEnabled(boolean sseEnabled) { + this.isRemoteStoreSSEnabled = sseEnabled; + } + public void setRemoteStoreRepository(String remoteStoreRepository) { this.remoteStoreRepository = remoteStoreRepository; } 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 0a365e4d756d8..12f9959d3fbff 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -5289,7 +5289,8 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { getThreadPool(), indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings, - indexSettings().isTranslogMetadataEnabled() + indexSettings().isTranslogMetadataEnabled(), + indexSettings().isRemoteStoreSSEnabled() ); } @@ -5312,7 +5313,8 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { shardId, indexSettings.getRemoteStorePathStrategy(), indexSettings().isTranslogMetadataEnabled(), - 0 + 0, + indexSettings.isRemoteStoreSSEnabled() ); } @@ -5321,7 +5323,8 @@ public void syncTranslogFilesFromGivenRemoteTranslog( ShardId shardId, RemoteStorePathStrategy remoteStorePathStrategy, boolean isTranslogMetadataEnabled, - long timestamp + long timestamp, + boolean isServerSideEncryptionEnabled ) throws IOException { RemoteFsTranslog.download( repository, @@ -5333,7 +5336,8 @@ public void syncTranslogFilesFromGivenRemoteTranslog( logger, shouldSeedRemoteStore(), isTranslogMetadataEnabled, - timestamp + timestamp, + isServerSideEncryptionEnabled ); } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 8f99efe502858..f769e9c0d08a6 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -422,7 +422,9 @@ void recoverFromSnapshotAndRemoteStore( remoteStoreRepository, indexUUID, shardId, - shallowCopyShardMetadata.getRemoteStorePathStrategy() + shallowCopyShardMetadata.getRemoteStorePathStrategy(), + null, + indexShard.indexSettings.isRemoteStoreSSEnabled() ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, @@ -503,7 +505,9 @@ void recoverShallowSnapshotV2( remoteSegmentStoreRepository, prevIndexMetadata.getIndexUUID(), shardId, - remoteStorePathStrategy + remoteStorePathStrategy, + null, + IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING.get(prevIndexMetadata.getSettings()) ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( recoverySource.pinnedTimestamp() @@ -523,7 +527,8 @@ void recoverShallowSnapshotV2( new ShardId(prevIndexMetadata.getIndex(), shardId.id()), remoteStorePathStrategy, RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), - recoverySource.pinnedTimestamp() + recoverySource.pinnedTimestamp(), + indexShard.indexSettings.isRemoteStoreSSEnabled() ); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 35aba694729cb..e08078af45f49 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -60,12 +60,12 @@ public RemoteSegmentStoreDirectoryFactory( public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException { String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); - return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy()); + return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy(), null, indexSettings.isRemoteStoreSSEnabled()); } public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId, RemoteStorePathStrategy pathStrategy) throws IOException { - return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null); + return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, false); } public Directory newDirectory( @@ -73,11 +73,13 @@ public Directory newDirectory( String indexUUID, ShardId shardId, RemoteStorePathStrategy pathStrategy, - String indexFixedPrefix + String indexFixedPrefix, + boolean serverSideEncryptionEnabled ) throws IOException { assert Objects.nonNull(pathStrategy); - try (Repository repository = repositoriesService.get().repository(repositoryName)) { - + // We should be not calling close for repository. + Repository repository = repositoriesService.get().repository(repositoryName); + try { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository); BlobPath repositoryBasePath = blobStoreRepository.basePath(); @@ -93,10 +95,12 @@ public Directory newDirectory( .fixedPrefix(segmentsPathFixedPrefix) .indexFixedPrefix(indexFixedPrefix) .build(); + + System.out.println("[PRANIT]: serverSideEncryptionEnabled value = " + serverSideEncryptionEnabled); // Derive the path for data directory of SEGMENTS BlobPath dataPath = pathStrategy.generatePath(dataPathInput); RemoteDirectory dataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore().blobContainer(dataPath), + blobStoreRepository.blobStore(serverSideEncryptionEnabled).blobContainer(dataPath), blobStoreRepository::maybeRateLimitRemoteUploadTransfers, blobStoreRepository::maybeRateLimitLowPriorityRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers, @@ -115,7 +119,7 @@ public Directory newDirectory( .build(); // Derive the path for metadata directory of SEGMENTS BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore().blobContainer(mdPath)); + RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore(serverSideEncryptionEnabled).blobContainer(mdPath)); // The path for lock is derived within the RemoteStoreLockManagerFactory RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager( diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java index b1742695b6748..08926c09fa033 100644 --- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java +++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java @@ -67,7 +67,8 @@ public static RemoteStoreMetadataLockManager newLockManager( String segmentsPathFixedPrefix, String indexFixedPrefix ) { - try (Repository repository = repositoriesService.repository(repositoryName)) { + Repository repository = repositoriesService.repository(repositoryName); + try { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobPath repositoryBasePath = ((BlobStoreRepository) repository).basePath(); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java index 1f2b2c48b471a..f67cde4d6c3ca 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java @@ -8,6 +8,7 @@ package org.opensearch.index.translog; +import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.repositories.RepositoriesService; @@ -37,12 +38,15 @@ public class RemoteBlobStoreInternalTranslogFactory implements TranslogFactory { private final RemoteStoreSettings remoteStoreSettings; + private final IndexSettings indexSettings; + public RemoteBlobStoreInternalTranslogFactory( Supplier repositoriesServiceSupplier, ThreadPool threadPool, String repositoryName, RemoteTranslogTransferTracker remoteTranslogTransferTracker, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + IndexSettings indexSettings ) { Repository repository; try { @@ -54,6 +58,7 @@ public RemoteBlobStoreInternalTranslogFactory( this.threadPool = threadPool; this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; this.remoteStoreSettings = remoteStoreSettings; + this.indexSettings = indexSettings; } @Override @@ -107,7 +112,8 @@ public Translog newTranslog( startedPrimarySupplier, remoteTranslogTransferTracker, remoteStoreSettings, - translogOperationHelper + translogOperationHelper, + indexSettings ); } else { return new RemoteFsTranslog( @@ -123,7 +129,8 @@ public Translog newTranslog( remoteTranslogTransferTracker, remoteStoreSettings, translogOperationHelper, - null + null, + indexSettings ); } } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java index 7fd915ba2c297..eb24214eef6a7 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java @@ -15,6 +15,7 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.Loggers; import org.opensearch.core.action.ActionListener; +import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.transfer.TranslogTransferManager; @@ -76,7 +77,8 @@ public RemoteFsTimestampAwareTranslog( BooleanSupplier startedPrimarySupplier, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings, - TranslogOperationHelper translogOperationHelper + TranslogOperationHelper translogOperationHelper, + IndexSettings indexSettings ) throws IOException { super( config, @@ -91,7 +93,8 @@ public RemoteFsTimestampAwareTranslog( remoteTranslogTransferTracker, remoteStoreSettings, translogOperationHelper, - null + null, + indexSettings ); logger = Loggers.getLogger(getClass(), shardId); this.metadataFilePinnedTimestampMap = new HashMap<>(); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index bbe8b739e2da4..2bb7492c3fce6 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lease.Releasable; @@ -19,6 +20,7 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; @@ -95,6 +97,7 @@ public class RemoteFsTranslog extends Translog { private final Semaphore syncPermit = new Semaphore(SYNC_PERMIT); protected final AtomicBoolean pauseSync = new AtomicBoolean(false); private final boolean isTranslogMetadataEnabled; + private IndexSettings indexSettings; public RemoteFsTranslog( TranslogConfig config, @@ -109,7 +112,8 @@ public RemoteFsTranslog( RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings, TranslogOperationHelper translogOperationHelper, - ChannelFactory channelFactory + ChannelFactory channelFactory, + IndexSettings indexSettings ) throws IOException { super( config, @@ -126,6 +130,8 @@ public RemoteFsTranslog( this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); + this.indexSettings = indexSettings; + boolean isServerSideEncryptionEnabled = this.indexSettings != null && this.indexSettings.isRemoteStoreSSEnabled(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -134,7 +140,8 @@ public RemoteFsTranslog( remoteTranslogTransferTracker, indexSettings().getRemoteStorePathStrategy(), remoteStoreSettings, - isTranslogMetadataEnabled + isTranslogMetadataEnabled, + isServerSideEncryptionEnabled ); try { if (config.downloadRemoteTranslogOnInit()) { @@ -193,7 +200,8 @@ public static void download( Logger logger, boolean seedRemote, boolean isTranslogMetadataEnabled, - long timestamp + long timestamp, + boolean isServerSideEncryptionEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, @@ -213,7 +221,8 @@ public static void download( remoteTranslogTransferTracker, pathStrategy, remoteStoreSettings, - isTranslogMetadataEnabled + isTranslogMetadataEnabled, + isServerSideEncryptionEnabled ); RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote, timestamp); logger.trace(remoteTranslogTransferTracker.toString()); @@ -325,7 +334,8 @@ public static TranslogTransferManager buildTranslogTransferManager( RemoteTranslogTransferTracker tracker, RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, - boolean isTranslogMetadataEnabled + boolean isTranslogMetadataEnabled, + boolean isServerSideEncryptionEnabled ) { assert Objects.nonNull(pathStrategy); String indexUUID = shardId.getIndex().getUUID(); @@ -348,7 +358,7 @@ public static TranslogTransferManager buildTranslogTransferManager( .fixedPrefix(remoteStoreSettings.getTranslogPathFixedPrefix()) .build(); BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool); + BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(isServerSideEncryptionEnabled), threadPool); return new TranslogTransferManager( shardId, transferService, @@ -655,7 +665,8 @@ public static void cleanup( ThreadPool threadPool, RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, - boolean isTranslogMetadataEnabled + boolean isTranslogMetadataEnabled, + boolean isServerSideEncryptionEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; @@ -671,7 +682,8 @@ public static void cleanup( remoteTranslogTransferTracker, pathStrategy, remoteStoreSettings, - isTranslogMetadataEnabled + isTranslogMetadataEnabled, + isServerSideEncryptionEnabled ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 667eb9edb65f7..bc715426030bd 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -728,7 +728,8 @@ private static BiFunction getTrans threadPool, indexSettings.getRemoteStoreTranslogRepository(), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), - remoteStoreSettings + remoteStoreSettings, + indexSettings ); } else if (RemoteStoreNodeAttribute.isTranslogRepoConfigured(settings) && shardRouting.primary()) { return new RemoteBlobStoreInternalTranslogFactory( @@ -736,7 +737,8 @@ private static BiFunction getTrans threadPool, RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo(indexSettings.getNodeSettings()), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), - remoteStoreSettings + remoteStoreSettings, + indexSettings ); } return new InternalTranslogFactory(); diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 56c3af3410643..55347220f2f6e 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -249,6 +249,16 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { return false; } + public static boolean isRemoteStoreMetadata(Settings settings) { + for (String metadataKey : settings.keySet()) { + System.out.println("metadataKey = " + metadataKey); + if(metadataKey.equals("sse_enabled")) { + return true; + } + } + return false; + } + public static boolean isRemoteDataAttributePresent(Settings settings) { return isSegmentRepoConfigured(settings) || isTranslogRepoConfigured(settings); } @@ -280,6 +290,10 @@ public static boolean isRemoteClusterStateConfigured(Settings settings) { return false; } + public static boolean isRemoteStoreServerSideEncryptionEnabled(Map repos) { + return true; + } + public static String getRemoteStoreSegmentRepo(Settings settings) { for (String prefix : REMOTE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEYS) { if (settings.get(Node.NODE_ATTRIBUTES.getKey() + prefix) != null) { diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java new file mode 100644 index 0000000000000..cb512276d1ad0 --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java @@ -0,0 +1,87 @@ +/* + * 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.repositories.blobstore; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.SetOnce; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.EncryptedBlobStore; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.repositories.RepositoryException; + +/** + * Provide for the BlobStore class + * + * @opensearch.internal + */ +public class BlobStoreProvider { + protected final Lifecycle lifecycle; + protected final RepositoryMetadata metadata; + protected final Object lock; + protected final BlobStoreRepository repository; + protected final SetOnce blobStore = new SetOnce<>(); + + public BlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { + this.lifecycle = lifecycle; + this.metadata = metadata; + this.lock = lock; + this.repository = repository; + } + + public BlobStore blobStore(boolean serverSideEncryption) { + return createBlobStore(blobStore, false); + } + + public BlobStore blobStore() { + // Assertion not true as Kraken threads use blobStore + return blobStore(false); + } + + protected BlobStore createBlobStore(SetOnce blobStore, boolean serverSideEncryptyion) { + // assertSnapshotOrGenericThread(); + BlobStore store = blobStore.get(); + if (store == null) { + synchronized (lock) { + store = blobStore.get(); + if (store == null) { + store = initBlobStore(serverSideEncryptyion); + if (!serverSideEncryptyion && metadata.cryptoMetadata() != null) { + store = new EncryptedBlobStore(store, metadata.cryptoMetadata()); + } + blobStore.set(store); + } + } + } + return store; + } + + public BlobStore getBlobStore(boolean serverSideEncryption) { + if (serverSideEncryption) { + throw new IllegalArgumentException("Provider Instance Type is not correct"); + } + return blobStore.get(); + } + + public BlobStore getBlobStore() { + return blobStore.get(); + } + + protected BlobStore initBlobStore(boolean serverSideEncryption) { + if (lifecycle.started() == false) { + throw new RepositoryException(metadata.name(), "repository is not in started state" + lifecycle.state()); + } + try { + return repository.createBlobStore(); + } catch (RepositoryException e) { + throw e; + } catch (Exception e) { + throw new RepositoryException(metadata.name(), "cannot create blob store", e); + } + } +} diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java new file mode 100644 index 0000000000000..72a6a0c56b4da --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java @@ -0,0 +1,48 @@ +/* + * 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.repositories.blobstore; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.SetOnce; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; + +/** + * Factory class for BlobStoreProvider. + */ +public class BlobStoreProviderFactory { + private final Lifecycle lifecycle; + private final RepositoryMetadata metadata; + private final Object lock; + private final BlobStoreRepository repository; + private static RemoteStoreBlobStoreProvider remoteStoreBlobStoreProvider; + private static BlobStoreProvider blobStoreProvider; + + public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { + this.lifecycle = lifecycle; + this.metadata = metadata; + this.lock = lock; + this.repository = repository; + } + + public BlobStoreProvider getBlobStoreProvider() { + if (RemoteStoreNodeAttribute.isRemoteStoreMetadata(metadata.settings())) { + if (remoteStoreBlobStoreProvider == null) { + remoteStoreBlobStoreProvider = new RemoteStoreBlobStoreProvider(repository, metadata, lifecycle, lock); + } + return remoteStoreBlobStoreProvider; + } else { + if (blobStoreProvider == null) { + blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); + } + return blobStoreProvider; + } + } +} diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 1b6aa3df2bc8a..958e535d0a4df 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -74,7 +74,6 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.DeleteResult; -import org.opensearch.common.blobstore.EncryptedBlobStore; import org.opensearch.common.blobstore.fs.FsBlobContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; import org.opensearch.common.blobstore.transfer.stream.RateLimitingOffsetRangeInputStream; @@ -567,7 +566,8 @@ protected static long calculateMaxWithinIntLimit(long defaultThresholdOfHeap, lo private final SetOnce snapshotShardPathBlobContainer = new SetOnce<>(); - private final SetOnce blobStore = new SetOnce<>(); +// private final SetOnce blobStore = new SetOnce<>(); + private final SetOnce blobStore = new SetOnce<>(); protected final ClusterService clusterService; @@ -683,10 +683,13 @@ protected void doStop() {} @Override protected void doClose() { - BlobStore store; + BlobStore store = null; // to close blobStore if blobStore initialization is started during close synchronized (lock) { - store = blobStore.get(); + BlobStoreProvider provider = blobStore.get(); + if (provider != null) { + store = provider.getBlobStore(false); + } } if (store != null) { try { @@ -983,7 +986,7 @@ public SetOnce getSnapshotShardPathBlobContainer() { // for test purposes only protected BlobStore getBlobStore() { - return blobStore.get(); + return blobStore.get().getBlobStore(false); } boolean getPrefixModeVerification() { @@ -1052,29 +1055,26 @@ protected BlobContainer snapshotShardPathBlobContainer() { * Public for testing. */ public BlobStore blobStore() { - BlobStore store = blobStore.get(); - if (store == null) { - synchronized (lock) { - store = blobStore.get(); - if (store == null) { - if (lifecycle.started() == false) { - throw new RepositoryException(metadata.name(), "repository is not in started state"); - } - try { - store = createBlobStore(); - if (metadata.cryptoMetadata() != null) { - store = new EncryptedBlobStore(store, metadata.cryptoMetadata()); - } - } catch (RepositoryException e) { - throw e; - } catch (Exception e) { - throw new RepositoryException(metadata.name(), "cannot create blob store", e); - } - blobStore.set(store); - } - } + return blobStore(false); + } + + /** + * Calls the existing blobStore() method. Specific repositories can implement the support for + * Server side encryption + * @param serverSideEncryption If the server side encryption is supported. + * @return BlobStore `Blobstore` for the repository + */ + public BlobStore blobStore(boolean serverSideEncryption) { + System.out.println("metadata = " + metadata); + BlobStoreProviderFactory providerFactory = new BlobStoreProviderFactory(this, metadata, lifecycle, lock); + BlobStoreProvider provider = providerFactory.getBlobStoreProvider(); + if (provider == null) { + System.out.println("provider is null "); + return null; } - return store; + System.out.println("provider = " + provider.getClass().getName()); + provider.blobStore(serverSideEncryption); + return provider.getBlobStore(serverSideEncryption); } /** @@ -1082,6 +1082,14 @@ public BlobStore blobStore() { */ protected abstract BlobStore createBlobStore() throws Exception; + protected BlobStore createServerSideEncryptedBlobStore() { + throw new UnsupportedOperationException(); + } + + protected BlobStore createClientSideEncryptedBlobStore() { + throw new UnsupportedOperationException(); + } + /** * Returns base path of the repository */ @@ -1122,7 +1130,12 @@ public Compressor getCompressor() { @Override public RepositoryStats stats() { - final BlobStore store = blobStore.get(); + BlobStoreProvider provider = blobStore.get(); + BlobStore store = null; + if (provider == null) { + return RepositoryStats.EMPTY_STATS; + } + store = provider.getBlobStore(false); if (store == null) { return RepositoryStats.EMPTY_STATS; } else if (store.extendedStats() != null && store.extendedStats().isEmpty() == false) { @@ -2394,7 +2407,8 @@ private void remoteTranslogCleanupAsync( remoteTranslogTransferTracker, remoteStorePathStrategy, remoteStoreSettings, - indexMetadataEnabled + indexMetadataEnabled, + false ); try { RemoteFsTimestampAwareTranslog.cleanupOfDeletedIndex(translogTransferManager, forceClean); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java new file mode 100644 index 0000000000000..a9ac47d24437b --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java @@ -0,0 +1,70 @@ +/* + * 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.repositories.blobstore; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.SetOnce; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.repositories.RepositoryException; + +/** + * BlobStoreProvider for RemoteStoreProvider + */ +public class RemoteStoreBlobStoreProvider extends BlobStoreProvider { + private final SetOnce serverSideEncryptedBlobStore = new SetOnce<>(); + + public RemoteStoreBlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { + super(repository, metadata, lifecycle, lock); + } + + public BlobStore getBlobStore(boolean serverSideEncryption) { + if (serverSideEncryption) { + return serverSideEncryptedBlobStore.get(); + } + return blobStore.get(); + } + + /** + * + */ + public BlobStore blobStore(boolean serverSideEncryption) { + System.out.println("serverSideEncryption = " + serverSideEncryption); + BlobStore store = null; + if (serverSideEncryption) { + store = serverSideEncryptedBlobStore.get(); + if (store == null) { + store = super.createBlobStore(serverSideEncryptedBlobStore, true); + } + } else { + store = super.blobStore(false); + } + return store; + } + + /** + * + */ + protected BlobStore initBlobStore(boolean serverSideEncryption) { + if (lifecycle.started() == false) { + throw new RepositoryException(metadata.name(), "repository is not in started state" + lifecycle.state()); + } + try { + if (serverSideEncryption) { + return repository.createServerSideEncryptedBlobStore(); + } else { + return repository.createClientSideEncryptedBlobStore(); + } + } catch (RepositoryException e) { + throw e; + } catch (Exception e) { + throw new RepositoryException(metadata.name(), "cannot create blob store", e); + } + } +} diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 29dd60c3e638f..533597ecf3c98 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -239,7 +239,8 @@ private IndexService newIndexService(IndexModule module) throws IOException { threadPool, indexSettings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 10), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + indexSettings ); } return new InternalTranslogFactory(); diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index edcdca3f7b3de..237d99b586467 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -202,6 +202,7 @@ protected RemoteFsTranslog createTranslogInstance( new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, + null, null ); } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 6d2bdbc8510f6..270eb4f9882b4 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -742,7 +742,8 @@ protected IndexShard newShard( threadPool, settings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 20), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + settings ); } return new InternalTranslogFactory(); From 8ffbee974937c278351f28bfe13f93e8122c6d29 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Fri, 10 Oct 2025 16:40:45 +0530 Subject: [PATCH 02/10] Use IndexSttings param instead of boolean Signed-off-by: Pranit Kumar --- .../TransportRemoteStoreMetadataAction.java | 4 ++-- .../org/opensearch/index/IndexService.java | 2 +- .../org/opensearch/index/IndexSettings.java | 16 +++++++-------- .../opensearch/index/shard/IndexShard.java | 8 ++++---- .../opensearch/index/shard/StoreRecovery.java | 8 +++++--- .../RemoteSegmentStoreDirectoryFactory.java | 11 +++++----- .../index/translog/RemoteFsTranslog.java | 16 +++++++-------- .../blobstore/BlobStoreProvider.java | 20 +++++++++++-------- .../blobstore/BlobStoreProviderFactory.java | 10 ++++------ .../blobstore/BlobStoreRepository.java | 19 +++++++++--------- ...deEncryptionEnabledBlobStoreProvider.java} | 20 +++++++++++-------- 11 files changed, 70 insertions(+), 64 deletions(-) rename server/src/main/java/org/opensearch/repositories/blobstore/{RemoteStoreBlobStoreProvider.java => ServerSideEncryptionEnabledBlobStoreProvider.java} (69%) 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 c57308b4bcca2..e4eab394c79a8 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 @@ -201,7 +201,7 @@ private Map> getSegmentMetadata( shardId, indexSettings.getRemoteStorePathStrategy(), null, - indexSettings.isRemoteStoreSSEnabled() + indexSettings ); Map segmentMetadataMapWithFilenames = remoteDirectory.readLatestNMetadataFiles(5); @@ -261,7 +261,7 @@ private Map> getTranslogMetadataFiles( indexSettings.getRemoteStorePathStrategy(), new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()), RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata), - indexSettings.isRemoteStoreSSEnabled() + indexSettings ); Map metadataMap = manager.readLatestNMetadataFiles(5); diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 762f30c2dad70..c5c2b3d1796cc 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -702,7 +702,7 @@ public synchronized IndexShard createShard( shardId, this.indexSettings.getRemoteStorePathStrategy(), this.indexSettings.getRemoteStoreSegmentPathPrefix(), - this.indexSettings.isRemoteStoreEnabled() + this.indexSettings ); } // When an instance of Store is created, a shardlock is created which is released on closing the instance of store. diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 8744848695f9f..b894e075eb91d 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -950,7 +950,7 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { */ private final boolean isCompositeIndex; - private boolean isRemoteStoreSSEnabled; + private boolean isServerSideEncryptionEnabled; /** * Denotes whether search via star tree index is enabled for this index @@ -1037,9 +1037,9 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null); replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); - isRemoteStoreSSEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false); + isServerSideEncryptionEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false); - System.out.println("[PRANIT]: IndexSettings.IndexSettings is for index " + this.index.getName() + " SSE ENABLED VALUE is " + isRemoteStoreSSEnabled); + System.out.println("[PRANIT]: IndexSettings.IndexSettings is for index " + this.index.getName() + " SSE ENABLED VALUE is " + isServerSideEncryptionEnabled); isWarmIndex = settings.getAsBoolean(IndexModule.IS_WARM_INDEX_SETTING.getKey(), false); @@ -1249,7 +1249,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(ALLOW_DERIVED_FIELDS, this::setAllowDerivedField); scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, this::setRemoteStoreEnabled); - scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, this::setRemoteStoreSseEnabled); + scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, this::setServerSideEncryptionEnabled); scopedSettings.addSettingsUpdateConsumer( IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, @@ -1442,8 +1442,8 @@ public boolean isRemoteStoreEnabled() { /** * Returns if remote store SSE is enabled for this index. */ - public boolean isRemoteStoreSSEnabled() { - return isRemoteStoreSSEnabled; + public boolean isServerSideEncryptionEnabled() { + return isServerSideEncryptionEnabled; } public boolean isAssignedOnRemoteNode() { @@ -2156,8 +2156,8 @@ public void setRemoteStoreEnabled(boolean isRemoteStoreEnabled) { this.isRemoteStoreEnabled = isRemoteStoreEnabled; } - public void setRemoteStoreSseEnabled(boolean sseEnabled) { - this.isRemoteStoreSSEnabled = sseEnabled; + public void setServerSideEncryptionEnabled(boolean sseEnabled) { + this.isServerSideEncryptionEnabled = sseEnabled; } public void setRemoteStoreRepository(String remoteStoreRepository) { 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 12f9959d3fbff..19ff152c38d81 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -5290,7 +5290,7 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings, indexSettings().isTranslogMetadataEnabled(), - indexSettings().isRemoteStoreSSEnabled() + indexSettings ); } @@ -5314,7 +5314,7 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), indexSettings().isTranslogMetadataEnabled(), 0, - indexSettings.isRemoteStoreSSEnabled() + indexSettings ); } @@ -5324,7 +5324,7 @@ public void syncTranslogFilesFromGivenRemoteTranslog( RemoteStorePathStrategy remoteStorePathStrategy, boolean isTranslogMetadataEnabled, long timestamp, - boolean isServerSideEncryptionEnabled + IndexSettings indexSettings ) throws IOException { RemoteFsTranslog.download( repository, @@ -5337,7 +5337,7 @@ public void syncTranslogFilesFromGivenRemoteTranslog( shouldSeedRemoteStore(), isTranslogMetadataEnabled, timestamp, - isServerSideEncryptionEnabled + indexSettings ); } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index f769e9c0d08a6..8c007d9def45d 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -56,6 +56,7 @@ import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.EngineException; import org.opensearch.index.mapper.MapperService; @@ -424,7 +425,7 @@ void recoverFromSnapshotAndRemoteStore( shardId, shallowCopyShardMetadata.getRemoteStorePathStrategy(), null, - indexShard.indexSettings.isRemoteStoreSSEnabled() + indexShard.indexSettings ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, @@ -500,6 +501,7 @@ void recoverShallowSnapshotV2( prevIndexMetadata.getSettings() ); } + IndexSettings indexSettings = new IndexSettings(prevIndexMetadata, prevIndexMetadata.getSettings()); RemoteStorePathStrategy remoteStorePathStrategy = RemoteStoreUtils.determineRemoteStorePathStrategy(prevIndexMetadata); RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( remoteSegmentStoreRepository, @@ -507,7 +509,7 @@ void recoverShallowSnapshotV2( shardId, remoteStorePathStrategy, null, - IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING.get(prevIndexMetadata.getSettings()) + indexSettings ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( recoverySource.pinnedTimestamp() @@ -528,7 +530,7 @@ void recoverShallowSnapshotV2( remoteStorePathStrategy, RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), recoverySource.pinnedTimestamp(), - indexShard.indexSettings.isRemoteStoreSSEnabled() + indexSettings ); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index e08078af45f49..e450cef3a1178 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -60,12 +60,12 @@ public RemoteSegmentStoreDirectoryFactory( public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException { String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); - return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy(), null, indexSettings.isRemoteStoreSSEnabled()); + return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy(), null, indexSettings); } public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId, RemoteStorePathStrategy pathStrategy) throws IOException { - return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, false); + return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, null); } public Directory newDirectory( @@ -74,7 +74,7 @@ public Directory newDirectory( ShardId shardId, RemoteStorePathStrategy pathStrategy, String indexFixedPrefix, - boolean serverSideEncryptionEnabled + IndexSettings indexSettings ) throws IOException { assert Objects.nonNull(pathStrategy); // We should be not calling close for repository. @@ -96,11 +96,10 @@ public Directory newDirectory( .indexFixedPrefix(indexFixedPrefix) .build(); - System.out.println("[PRANIT]: serverSideEncryptionEnabled value = " + serverSideEncryptionEnabled); // Derive the path for data directory of SEGMENTS BlobPath dataPath = pathStrategy.generatePath(dataPathInput); RemoteDirectory dataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore(serverSideEncryptionEnabled).blobContainer(dataPath), + blobStoreRepository.blobStore(indexSettings).blobContainer(dataPath), blobStoreRepository::maybeRateLimitRemoteUploadTransfers, blobStoreRepository::maybeRateLimitLowPriorityRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers, @@ -119,7 +118,7 @@ public Directory newDirectory( .build(); // Derive the path for metadata directory of SEGMENTS BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore(serverSideEncryptionEnabled).blobContainer(mdPath)); + RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore(indexSettings).blobContainer(mdPath)); // The path for lock is derived within the RemoteStoreLockManagerFactory RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager( diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 2bb7492c3fce6..22be3b38beacc 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lease.Releasable; @@ -131,7 +130,6 @@ public RemoteFsTranslog( fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); this.indexSettings = indexSettings; - boolean isServerSideEncryptionEnabled = this.indexSettings != null && this.indexSettings.isRemoteStoreSSEnabled(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -141,7 +139,7 @@ public RemoteFsTranslog( indexSettings().getRemoteStorePathStrategy(), remoteStoreSettings, isTranslogMetadataEnabled, - isServerSideEncryptionEnabled + this.indexSettings ); try { if (config.downloadRemoteTranslogOnInit()) { @@ -201,7 +199,7 @@ public static void download( boolean seedRemote, boolean isTranslogMetadataEnabled, long timestamp, - boolean isServerSideEncryptionEnabled + IndexSettings indexSettings ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, @@ -222,7 +220,7 @@ public static void download( pathStrategy, remoteStoreSettings, isTranslogMetadataEnabled, - isServerSideEncryptionEnabled + indexSettings ); RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote, timestamp); logger.trace(remoteTranslogTransferTracker.toString()); @@ -335,7 +333,7 @@ public static TranslogTransferManager buildTranslogTransferManager( RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, boolean isTranslogMetadataEnabled, - boolean isServerSideEncryptionEnabled + IndexSettings indexSettings ) { assert Objects.nonNull(pathStrategy); String indexUUID = shardId.getIndex().getUUID(); @@ -358,7 +356,7 @@ public static TranslogTransferManager buildTranslogTransferManager( .fixedPrefix(remoteStoreSettings.getTranslogPathFixedPrefix()) .build(); BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(isServerSideEncryptionEnabled), threadPool); + BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(indexSettings), threadPool); return new TranslogTransferManager( shardId, transferService, @@ -666,7 +664,7 @@ public static void cleanup( RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, boolean isTranslogMetadataEnabled, - boolean isServerSideEncryptionEnabled + IndexSettings indexSettings ) throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; @@ -683,7 +681,7 @@ public static void cleanup( pathStrategy, remoteStoreSettings, isTranslogMetadataEnabled, - isServerSideEncryptionEnabled + indexSettings ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java index cb512276d1ad0..85a87cee4be21 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java @@ -13,8 +13,11 @@ import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.EncryptedBlobStore; import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.index.IndexSettings; import org.opensearch.repositories.RepositoryException; +import java.util.function.Supplier; + /** * Provide for the BlobStore class * @@ -34,24 +37,25 @@ public BlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata meta this.repository = repository; } - public BlobStore blobStore(boolean serverSideEncryption) { - return createBlobStore(blobStore, false); + protected BlobStore blobStore(IndexSettings indexSettings) { + boolean serverSideEncryption = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); + return createBlobStore(blobStore, serverSideEncryption); } public BlobStore blobStore() { // Assertion not true as Kraken threads use blobStore - return blobStore(false); + return blobStore(null); } - protected BlobStore createBlobStore(SetOnce blobStore, boolean serverSideEncryptyion) { + protected BlobStore createBlobStore(SetOnce blobStore, boolean serverSideEncryption) { // assertSnapshotOrGenericThread(); BlobStore store = blobStore.get(); if (store == null) { synchronized (lock) { store = blobStore.get(); if (store == null) { - store = initBlobStore(serverSideEncryptyion); - if (!serverSideEncryptyion && metadata.cryptoMetadata() != null) { + store = initBlobStore(serverSideEncryption); + if (!serverSideEncryption && metadata.cryptoMetadata() != null) { store = new EncryptedBlobStore(store, metadata.cryptoMetadata()); } blobStore.set(store); @@ -61,8 +65,8 @@ protected BlobStore createBlobStore(SetOnce blobStore, boolean server return store; } - public BlobStore getBlobStore(boolean serverSideEncryption) { - if (serverSideEncryption) { + public BlobStore getBlobStore(IndexSettings indexSettings) { + if (indexSettings != null && indexSettings.isServerSideEncryptionEnabled()) { throw new IllegalArgumentException("Provider Instance Type is not correct"); } return blobStore.get(); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java index 72a6a0c56b4da..4981c09ba4617 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java @@ -9,8 +9,6 @@ package org.opensearch.repositories.blobstore; import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.SetOnce; -import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -22,7 +20,7 @@ public class BlobStoreProviderFactory { private final RepositoryMetadata metadata; private final Object lock; private final BlobStoreRepository repository; - private static RemoteStoreBlobStoreProvider remoteStoreBlobStoreProvider; + private static ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider; private static BlobStoreProvider blobStoreProvider; public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { @@ -34,10 +32,10 @@ public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetada public BlobStoreProvider getBlobStoreProvider() { if (RemoteStoreNodeAttribute.isRemoteStoreMetadata(metadata.settings())) { - if (remoteStoreBlobStoreProvider == null) { - remoteStoreBlobStoreProvider = new RemoteStoreBlobStoreProvider(repository, metadata, lifecycle, lock); + if (serverSideEncryptionEnabledBlobStoreProvider == null) { + serverSideEncryptionEnabledBlobStoreProvider = new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); } - return remoteStoreBlobStoreProvider; + return serverSideEncryptionEnabledBlobStoreProvider; } else { if (blobStoreProvider == null) { blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 958e535d0a4df..5db292a2e3c53 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -108,6 +108,7 @@ import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; @@ -688,7 +689,7 @@ protected void doClose() { synchronized (lock) { BlobStoreProvider provider = blobStore.get(); if (provider != null) { - store = provider.getBlobStore(false); + store = provider.getBlobStore(null); } } if (store != null) { @@ -986,7 +987,7 @@ public SetOnce getSnapshotShardPathBlobContainer() { // for test purposes only protected BlobStore getBlobStore() { - return blobStore.get().getBlobStore(false); + return blobStore.get().getBlobStore(null); } boolean getPrefixModeVerification() { @@ -1055,16 +1056,16 @@ protected BlobContainer snapshotShardPathBlobContainer() { * Public for testing. */ public BlobStore blobStore() { - return blobStore(false); + return blobStore(null); } /** * Calls the existing blobStore() method. Specific repositories can implement the support for * Server side encryption - * @param serverSideEncryption If the server side encryption is supported. + * @param indexSettings IndexSetting. * @return BlobStore `Blobstore` for the repository */ - public BlobStore blobStore(boolean serverSideEncryption) { + public BlobStore blobStore(IndexSettings indexSettings) { System.out.println("metadata = " + metadata); BlobStoreProviderFactory providerFactory = new BlobStoreProviderFactory(this, metadata, lifecycle, lock); BlobStoreProvider provider = providerFactory.getBlobStoreProvider(); @@ -1073,8 +1074,8 @@ public BlobStore blobStore(boolean serverSideEncryption) { return null; } System.out.println("provider = " + provider.getClass().getName()); - provider.blobStore(serverSideEncryption); - return provider.getBlobStore(serverSideEncryption); + provider.blobStore(indexSettings); + return provider.getBlobStore(indexSettings); } /** @@ -1135,7 +1136,7 @@ public RepositoryStats stats() { if (provider == null) { return RepositoryStats.EMPTY_STATS; } - store = provider.getBlobStore(false); + store = provider.getBlobStore(null); if (store == null) { return RepositoryStats.EMPTY_STATS; } else if (store.extendedStats() != null && store.extendedStats().isEmpty() == false) { @@ -2408,7 +2409,7 @@ private void remoteTranslogCleanupAsync( remoteStorePathStrategy, remoteStoreSettings, indexMetadataEnabled, - false + null ); try { RemoteFsTimestampAwareTranslog.cleanupOfDeletedIndex(translogTransferManager, forceClean); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java similarity index 69% rename from server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java rename to server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java index a9ac47d24437b..4d57dfb6683b9 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/RemoteStoreBlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java @@ -12,20 +12,23 @@ import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.index.IndexSettings; import org.opensearch.repositories.RepositoryException; /** * BlobStoreProvider for RemoteStoreProvider */ -public class RemoteStoreBlobStoreProvider extends BlobStoreProvider { +public class ServerSideEncryptionEnabledBlobStoreProvider extends BlobStoreProvider { private final SetOnce serverSideEncryptedBlobStore = new SetOnce<>(); - public RemoteStoreBlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { + public ServerSideEncryptionEnabledBlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { super(repository, metadata, lifecycle, lock); } - public BlobStore getBlobStore(boolean serverSideEncryption) { - if (serverSideEncryption) { + public BlobStore getBlobStore(IndexSettings indexSettings) { + boolean serverSideEncryptionEnabled = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); + + if (serverSideEncryptionEnabled) { return serverSideEncryptedBlobStore.get(); } return blobStore.get(); @@ -34,16 +37,17 @@ public BlobStore getBlobStore(boolean serverSideEncryption) { /** * */ - public BlobStore blobStore(boolean serverSideEncryption) { - System.out.println("serverSideEncryption = " + serverSideEncryption); + public BlobStore blobStore(IndexSettings indexSettings) { + boolean serverSideEncryptionEnabled = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); + System.out.println("serverSideEncryption = " + serverSideEncryptionEnabled); BlobStore store = null; - if (serverSideEncryption) { + if (serverSideEncryptionEnabled) { store = serverSideEncryptedBlobStore.get(); if (store == null) { store = super.createBlobStore(serverSideEncryptedBlobStore, true); } } else { - store = super.blobStore(false); + store = super.blobStore(indexSettings); } return store; } From aefe2baab08ff5cc56ed8dd2026e4eb1efc273b5 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Tue, 14 Oct 2025 17:30:30 +0530 Subject: [PATCH 03/10] Add Bloblevel support for serversideencryption. Signed-off-by: Pranit Kumar --- .../TransportRemoteStoreMetadataAction.java | 5 +- .../org/opensearch/index/IndexService.java | 2 +- .../org/opensearch/index/IndexSettings.java | 8 +-- .../opensearch/index/shard/IndexShard.java | 8 +-- .../opensearch/index/shard/StoreRecovery.java | 6 +- .../RemoteSegmentStoreDirectoryFactory.java | 19 ++++-- ...emoteBlobStoreInternalTranslogFactory.java | 11 ++- .../RemoteFsTimestampAwareTranslog.java | 5 +- .../index/translog/RemoteFsTranslog.java | 24 ++++--- .../opensearch/indices/IndicesService.java | 4 +- .../remotestore/RemoteStoreNodeAttribute.java | 3 +- .../blobstore/BlobStoreProvider.java | 35 ++++++---- .../blobstore/BlobStoreProviderFactory.java | 20 +++--- .../blobstore/BlobStoreRepository.java | 68 +++++++++++-------- ...ideEncryptionEnabledBlobStoreProvider.java | 38 +++++++---- .../opensearch/index/IndexModuleTests.java | 2 +- ...moteSegmentStoreDirectoryFactoryTests.java | 3 +- .../RemoteFsTimestampAwareTranslogTests.java | 6 +- .../index/translog/RemoteFsTranslogTests.java | 11 +-- .../index/shard/IndexShardTestCase.java | 2 +- 20 files changed, 165 insertions(+), 115 deletions(-) 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 e4eab394c79a8..e0773130653fd 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 @@ -109,7 +109,6 @@ protected void doExecute(Task task, RemoteStoreMetadataRequest request, ActionLi Index index = indexMetadata.getIndex(); IndexSettings indexSettings = new IndexSettings(indexMetadata, clusterService.getSettings()); - int[] shardIds = request.shards().length == 0 ? java.util.stream.IntStream.range(0, indexMetadata.getNumberOfShards()).toArray() : Arrays.stream(request.shards()).mapToInt(Integer::parseInt).toArray(); @@ -201,7 +200,7 @@ private Map> getSegmentMetadata( shardId, indexSettings.getRemoteStorePathStrategy(), null, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); Map segmentMetadataMapWithFilenames = remoteDirectory.readLatestNMetadataFiles(5); @@ -261,7 +260,7 @@ private Map> getTranslogMetadataFiles( indexSettings.getRemoteStorePathStrategy(), new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()), RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata), - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); Map metadataMap = manager.readLatestNMetadataFiles(5); diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index c5c2b3d1796cc..d73f8f5e5d9e8 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -702,7 +702,7 @@ public synchronized IndexShard createShard( shardId, this.indexSettings.getRemoteStorePathStrategy(), this.indexSettings.getRemoteStoreSegmentPathPrefix(), - this.indexSettings + this.indexSettings.isServerSideEncryptionEnabled() ); } // When an instance of Store is created, a shardlock is created which is released on closing the instance of store. diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index b894e075eb91d..7f27aea2a1447 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -1038,9 +1038,6 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); isServerSideEncryptionEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false); - - System.out.println("[PRANIT]: IndexSettings.IndexSettings is for index " + this.index.getName() + " SSE ENABLED VALUE is " + isServerSideEncryptionEnabled); - isWarmIndex = settings.getAsBoolean(IndexModule.IS_WARM_INDEX_SETTING.getKey(), false); remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); @@ -1249,7 +1246,10 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(ALLOW_DERIVED_FIELDS, this::setAllowDerivedField); scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, this::setRemoteStoreEnabled); - scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, this::setServerSideEncryptionEnabled); + scopedSettings.addSettingsUpdateConsumer( + IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, + this::setServerSideEncryptionEnabled + ); scopedSettings.addSettingsUpdateConsumer( IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, 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 19ff152c38d81..02e9ef196e35f 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -5290,7 +5290,7 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings, indexSettings().isTranslogMetadataEnabled(), - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); } @@ -5314,7 +5314,7 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), indexSettings().isTranslogMetadataEnabled(), 0, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); } @@ -5324,7 +5324,7 @@ public void syncTranslogFilesFromGivenRemoteTranslog( RemoteStorePathStrategy remoteStorePathStrategy, boolean isTranslogMetadataEnabled, long timestamp, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { RemoteFsTranslog.download( repository, @@ -5337,7 +5337,7 @@ public void syncTranslogFilesFromGivenRemoteTranslog( shouldSeedRemoteStore(), isTranslogMetadataEnabled, timestamp, - indexSettings + isServerSideEncryptionEnabled ); } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 8c007d9def45d..50e4ce70cc0d5 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -425,7 +425,7 @@ void recoverFromSnapshotAndRemoteStore( shardId, shallowCopyShardMetadata.getRemoteStorePathStrategy(), null, - indexShard.indexSettings + indexShard.indexSettings.isServerSideEncryptionEnabled() ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, @@ -509,7 +509,7 @@ void recoverShallowSnapshotV2( shardId, remoteStorePathStrategy, null, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( recoverySource.pinnedTimestamp() @@ -530,7 +530,7 @@ void recoverShallowSnapshotV2( remoteStorePathStrategy, RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), recoverySource.pinnedTimestamp(), - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index e450cef3a1178..4704ed88e2cb7 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -60,12 +60,19 @@ public RemoteSegmentStoreDirectoryFactory( public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException { String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); - return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy(), null, indexSettings); + return newDirectory( + repositoryName, + indexUUID, + path.getShardId(), + indexSettings.getRemoteStorePathStrategy(), + null, + indexSettings.isServerSideEncryptionEnabled() + ); } public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId, RemoteStorePathStrategy pathStrategy) throws IOException { - return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, null); + return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, false); } public Directory newDirectory( @@ -74,7 +81,7 @@ public Directory newDirectory( ShardId shardId, RemoteStorePathStrategy pathStrategy, String indexFixedPrefix, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { assert Objects.nonNull(pathStrategy); // We should be not calling close for repository. @@ -99,7 +106,7 @@ public Directory newDirectory( // Derive the path for data directory of SEGMENTS BlobPath dataPath = pathStrategy.generatePath(dataPathInput); RemoteDirectory dataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore(indexSettings).blobContainer(dataPath), + blobStoreRepository.blobStore(isServerSideEncryptionEnabled).blobContainer(dataPath), blobStoreRepository::maybeRateLimitRemoteUploadTransfers, blobStoreRepository::maybeRateLimitLowPriorityRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers, @@ -118,7 +125,9 @@ public Directory newDirectory( .build(); // Derive the path for metadata directory of SEGMENTS BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore(indexSettings).blobContainer(mdPath)); + RemoteDirectory metadataDirectory = new RemoteDirectory( + blobStoreRepository.blobStore(isServerSideEncryptionEnabled).blobContainer(mdPath) + ); // The path for lock is derived within the RemoteStoreLockManagerFactory RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager( diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java index f67cde4d6c3ca..501dbe2962d29 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java @@ -8,7 +8,6 @@ package org.opensearch.index.translog; -import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.repositories.RepositoriesService; @@ -38,7 +37,7 @@ public class RemoteBlobStoreInternalTranslogFactory implements TranslogFactory { private final RemoteStoreSettings remoteStoreSettings; - private final IndexSettings indexSettings; + private final boolean isServerSideEncryptionEnabled; public RemoteBlobStoreInternalTranslogFactory( Supplier repositoriesServiceSupplier, @@ -46,7 +45,7 @@ public RemoteBlobStoreInternalTranslogFactory( String repositoryName, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) { Repository repository; try { @@ -58,7 +57,7 @@ public RemoteBlobStoreInternalTranslogFactory( this.threadPool = threadPool; this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; this.remoteStoreSettings = remoteStoreSettings; - this.indexSettings = indexSettings; + this.isServerSideEncryptionEnabled = isServerSideEncryptionEnabled; } @Override @@ -113,7 +112,7 @@ public Translog newTranslog( remoteTranslogTransferTracker, remoteStoreSettings, translogOperationHelper, - indexSettings + isServerSideEncryptionEnabled ); } else { return new RemoteFsTranslog( @@ -130,7 +129,7 @@ public Translog newTranslog( remoteStoreSettings, translogOperationHelper, null, - indexSettings + isServerSideEncryptionEnabled ); } } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java index eb24214eef6a7..1832d1e7d035a 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java @@ -15,7 +15,6 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.Loggers; import org.opensearch.core.action.ActionListener; -import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.transfer.TranslogTransferManager; @@ -78,7 +77,7 @@ public RemoteFsTimestampAwareTranslog( RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings, TranslogOperationHelper translogOperationHelper, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { super( config, @@ -94,7 +93,7 @@ public RemoteFsTimestampAwareTranslog( remoteStoreSettings, translogOperationHelper, null, - indexSettings + isServerSideEncryptionEnabled ); logger = Loggers.getLogger(getClass(), shardId); this.metadataFilePinnedTimestampMap = new HashMap<>(); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 22be3b38beacc..1e3415f4044a6 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -19,7 +19,6 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; -import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; @@ -96,7 +95,7 @@ public class RemoteFsTranslog extends Translog { private final Semaphore syncPermit = new Semaphore(SYNC_PERMIT); protected final AtomicBoolean pauseSync = new AtomicBoolean(false); private final boolean isTranslogMetadataEnabled; - private IndexSettings indexSettings; + private final boolean isServerSideEncryptionEnabled; public RemoteFsTranslog( TranslogConfig config, @@ -112,7 +111,7 @@ public RemoteFsTranslog( RemoteStoreSettings remoteStoreSettings, TranslogOperationHelper translogOperationHelper, ChannelFactory channelFactory, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { super( config, @@ -129,7 +128,7 @@ public RemoteFsTranslog( this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); - this.indexSettings = indexSettings; + this.isServerSideEncryptionEnabled = isServerSideEncryptionEnabled; this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -139,7 +138,7 @@ public RemoteFsTranslog( indexSettings().getRemoteStorePathStrategy(), remoteStoreSettings, isTranslogMetadataEnabled, - this.indexSettings + isServerSideEncryptionEnabled ); try { if (config.downloadRemoteTranslogOnInit()) { @@ -199,7 +198,7 @@ public static void download( boolean seedRemote, boolean isTranslogMetadataEnabled, long timestamp, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, @@ -220,7 +219,7 @@ public static void download( pathStrategy, remoteStoreSettings, isTranslogMetadataEnabled, - indexSettings + isServerSideEncryptionEnabled ); RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote, timestamp); logger.trace(remoteTranslogTransferTracker.toString()); @@ -333,7 +332,7 @@ public static TranslogTransferManager buildTranslogTransferManager( RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, boolean isTranslogMetadataEnabled, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) { assert Objects.nonNull(pathStrategy); String indexUUID = shardId.getIndex().getUUID(); @@ -356,7 +355,10 @@ public static TranslogTransferManager buildTranslogTransferManager( .fixedPrefix(remoteStoreSettings.getTranslogPathFixedPrefix()) .build(); BlobPath mdPath = pathStrategy.generatePath(mdPathInput); - BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(indexSettings), threadPool); + BlobStoreTransferService transferService = new BlobStoreTransferService( + blobStoreRepository.blobStore(isServerSideEncryptionEnabled), + threadPool + ); return new TranslogTransferManager( shardId, transferService, @@ -664,7 +666,7 @@ public static void cleanup( RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, boolean isTranslogMetadataEnabled, - IndexSettings indexSettings + boolean isServerSideEncryptionEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; @@ -681,7 +683,7 @@ public static void cleanup( pathStrategy, remoteStoreSettings, isTranslogMetadataEnabled, - indexSettings + isServerSideEncryptionEnabled ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index bc715426030bd..6bb40fcd908b8 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -729,7 +729,7 @@ private static BiFunction getTrans indexSettings.getRemoteStoreTranslogRepository(), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), remoteStoreSettings, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); } else if (RemoteStoreNodeAttribute.isTranslogRepoConfigured(settings) && shardRouting.primary()) { return new RemoteBlobStoreInternalTranslogFactory( @@ -738,7 +738,7 @@ private static BiFunction getTrans RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo(indexSettings.getNodeSettings()), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), remoteStoreSettings, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); } return new InternalTranslogFactory(); diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 55347220f2f6e..bb824c736a927 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -251,8 +251,7 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { public static boolean isRemoteStoreMetadata(Settings settings) { for (String metadataKey : settings.keySet()) { - System.out.println("metadataKey = " + metadataKey); - if(metadataKey.equals("sse_enabled")) { + if (metadataKey.equals("sse_enabled")) { return true; } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java index 85a87cee4be21..a22158c3c945f 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java @@ -8,22 +8,22 @@ package org.opensearch.repositories.blobstore; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.EncryptedBlobStore; import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.index.IndexSettings; import org.opensearch.repositories.RepositoryException; -import java.util.function.Supplier; - /** * Provide for the BlobStore class * * @opensearch.internal */ public class BlobStoreProvider { + private static final Logger logger = LogManager.getLogger(BlobStoreProvider.class); protected final Lifecycle lifecycle; protected final RepositoryMetadata metadata; protected final Object lock; @@ -37,19 +37,19 @@ public BlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata meta this.repository = repository; } - protected BlobStore blobStore(IndexSettings indexSettings) { - boolean serverSideEncryption = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); - return createBlobStore(blobStore, serverSideEncryption); + protected BlobStore blobStore(boolean serverSideEncryptionEnabled) { + return createBlobStore(blobStore, serverSideEncryptionEnabled); } public BlobStore blobStore() { // Assertion not true as Kraken threads use blobStore - return blobStore(null); + return blobStore(false); } protected BlobStore createBlobStore(SetOnce blobStore, boolean serverSideEncryption) { // assertSnapshotOrGenericThread(); BlobStore store = blobStore.get(); + logger.info("1.store = " + store); if (store == null) { synchronized (lock) { store = blobStore.get(); @@ -62,19 +62,20 @@ protected BlobStore createBlobStore(SetOnce blobStore, boolean server } } } + logger.info("2.store = " + store); return store; } - public BlobStore getBlobStore(IndexSettings indexSettings) { - if (indexSettings != null && indexSettings.isServerSideEncryptionEnabled()) { + public BlobStore getBlobStore(boolean serverSideEncryptionEnabled) { + if (serverSideEncryptionEnabled) { throw new IllegalArgumentException("Provider Instance Type is not correct"); } return blobStore.get(); } - public BlobStore getBlobStore() { - return blobStore.get(); - } + // public BlobStore getBlobStore() { + // return blobStore.get(); + // } protected BlobStore initBlobStore(boolean serverSideEncryption) { if (lifecycle.started() == false) { @@ -88,4 +89,14 @@ protected BlobStore initBlobStore(boolean serverSideEncryption) { throw new RepositoryException(metadata.name(), "cannot create blob store", e); } } + + public void close() { + try { + if (blobStore.get() != null) { + blobStore.get().close(); + } + } catch (Exception t) { + logger.warn("cannot close blob store", t); + } + } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java index 4981c09ba4617..409aad44d03e3 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java @@ -9,6 +9,7 @@ package org.opensearch.repositories.blobstore; import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.SetOnce; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -20,8 +21,8 @@ public class BlobStoreProviderFactory { private final RepositoryMetadata metadata; private final Object lock; private final BlobStoreRepository repository; - private static ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider; - private static BlobStoreProvider blobStoreProvider; + private final SetOnce serverSideEncryptionEnabledBlobStoreProvider = new SetOnce<>(); + private final SetOnce blobStoreProvider = new SetOnce<>(); public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { this.lifecycle = lifecycle; @@ -32,15 +33,18 @@ public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetada public BlobStoreProvider getBlobStoreProvider() { if (RemoteStoreNodeAttribute.isRemoteStoreMetadata(metadata.settings())) { - if (serverSideEncryptionEnabledBlobStoreProvider == null) { - serverSideEncryptionEnabledBlobStoreProvider = new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); + if (serverSideEncryptionEnabledBlobStoreProvider.get() == null) { + ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider = + new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); + this.serverSideEncryptionEnabledBlobStoreProvider.set(serverSideEncryptionEnabledBlobStoreProvider); } - return serverSideEncryptionEnabledBlobStoreProvider; + return serverSideEncryptionEnabledBlobStoreProvider.get(); } else { - if (blobStoreProvider == null) { - blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); + if (blobStoreProvider.get() == null) { + BlobStoreProvider blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); + this.blobStoreProvider.set(blobStoreProvider); } - return blobStoreProvider; + return blobStoreProvider.get(); } } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 5db292a2e3c53..888245623c697 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -108,7 +108,6 @@ import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; @@ -567,8 +566,8 @@ protected static long calculateMaxWithinIntLimit(long defaultThresholdOfHeap, lo private final SetOnce snapshotShardPathBlobContainer = new SetOnce<>(); -// private final SetOnce blobStore = new SetOnce<>(); - private final SetOnce blobStore = new SetOnce<>(); + private final SetOnce blobStoreProvideFactory = new SetOnce<>(); + private final SetOnce blobStoreProvider = new SetOnce<>(); protected final ClusterService clusterService; @@ -684,22 +683,21 @@ protected void doStop() {} @Override protected void doClose() { - BlobStore store = null; + BlobStoreProvider provider = null; // to close blobStore if blobStore initialization is started during close synchronized (lock) { - BlobStoreProvider provider = blobStore.get(); + provider = blobStoreProvider.get(); + // Moving this inside synchronized block if (provider != null) { - store = provider.getBlobStore(null); - } - } - if (store != null) { - try { - closed = true; - store.close(); - } catch (Exception t) { - logger.warn("cannot close blob store", t); + try { + provider.close(); + closed = true; + } catch (Exception t) { + logger.warn("cannot close blob store", t); + } } } + } @Override @@ -987,7 +985,7 @@ public SetOnce getSnapshotShardPathBlobContainer() { // for test purposes only protected BlobStore getBlobStore() { - return blobStore.get().getBlobStore(null); + return blobStoreProvider.get() != null ? blobStoreProvider.get().getBlobStore(false) : null; } boolean getPrefixModeVerification() { @@ -1056,26 +1054,36 @@ protected BlobContainer snapshotShardPathBlobContainer() { * Public for testing. */ public BlobStore blobStore() { - return blobStore(null); + return blobStore(false); } /** * Calls the existing blobStore() method. Specific repositories can implement the support for * Server side encryption - * @param indexSettings IndexSetting. + * @param serverSideEncryptionEnabled ServerSideEncryptionEnabled Value. * @return BlobStore `Blobstore` for the repository */ - public BlobStore blobStore(IndexSettings indexSettings) { - System.out.println("metadata = " + metadata); - BlobStoreProviderFactory providerFactory = new BlobStoreProviderFactory(this, metadata, lifecycle, lock); - BlobStoreProvider provider = providerFactory.getBlobStoreProvider(); + public BlobStore blobStore(boolean serverSideEncryptionEnabled) { + BlobStoreProviderFactory providerFactory = this.blobStoreProvideFactory.get(); + logger.info("providerFactory = " + providerFactory); + if (providerFactory == null) { + synchronized (lock) { + providerFactory = new BlobStoreProviderFactory(this, metadata, lifecycle, lock); + blobStoreProvideFactory.set(providerFactory); + } + } + logger.info("2.providerFactory = " + providerFactory); + + BlobStoreProvider provider = this.blobStoreProvider.get(); + logger.info("1.provider = " + provider); if (provider == null) { - System.out.println("provider is null "); - return null; + synchronized (lock) { + provider = providerFactory.getBlobStoreProvider(); + this.blobStoreProvider.set(provider); + } } - System.out.println("provider = " + provider.getClass().getName()); - provider.blobStore(indexSettings); - return provider.getBlobStore(indexSettings); + logger.info("2.provider = " + provider); + return provider.blobStore(serverSideEncryptionEnabled); } /** @@ -1131,12 +1139,12 @@ public Compressor getCompressor() { @Override public RepositoryStats stats() { - BlobStoreProvider provider = blobStore.get(); + BlobStoreProvider provider = blobStoreProvider.get(); BlobStore store = null; if (provider == null) { return RepositoryStats.EMPTY_STATS; } - store = provider.getBlobStore(null); + store = provider.getBlobStore(false); if (store == null) { return RepositoryStats.EMPTY_STATS; } else if (store.extendedStats() != null && store.extendedStats().isEmpty() == false) { @@ -2409,7 +2417,7 @@ private void remoteTranslogCleanupAsync( remoteStorePathStrategy, remoteStoreSettings, indexMetadataEnabled, - null + false ); try { RemoteFsTimestampAwareTranslog.cleanupOfDeletedIndex(translogTransferManager, forceClean); @@ -4533,7 +4541,7 @@ public void verify(String seed, DiscoveryNode localNode) { @Override public String toString() { - return "BlobStoreRepository[" + "[" + metadata.name() + "], [" + blobStore.get() + ']' + ']'; + return "BlobStoreRepository[" + "[" + metadata.name() + "], [" + blobStoreProvider.get() + ']' + ']'; } /** diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java index 4d57dfb6683b9..7181e6f07118b 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java @@ -8,26 +8,31 @@ package org.opensearch.repositories.blobstore; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.index.IndexSettings; import org.opensearch.repositories.RepositoryException; /** * BlobStoreProvider for RemoteStoreProvider */ public class ServerSideEncryptionEnabledBlobStoreProvider extends BlobStoreProvider { + private static final Logger logger = LogManager.getLogger(ServerSideEncryptionEnabledBlobStoreProvider.class); private final SetOnce serverSideEncryptedBlobStore = new SetOnce<>(); - public ServerSideEncryptionEnabledBlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { + public ServerSideEncryptionEnabledBlobStoreProvider( + BlobStoreRepository repository, + RepositoryMetadata metadata, + Lifecycle lifecycle, + Object lock + ) { super(repository, metadata, lifecycle, lock); } - public BlobStore getBlobStore(IndexSettings indexSettings) { - boolean serverSideEncryptionEnabled = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); - + public BlobStore getBlobStore(boolean serverSideEncryptionEnabled) { if (serverSideEncryptionEnabled) { return serverSideEncryptedBlobStore.get(); } @@ -37,9 +42,7 @@ public BlobStore getBlobStore(IndexSettings indexSettings) { /** * */ - public BlobStore blobStore(IndexSettings indexSettings) { - boolean serverSideEncryptionEnabled = indexSettings != null && indexSettings.isServerSideEncryptionEnabled(); - System.out.println("serverSideEncryption = " + serverSideEncryptionEnabled); + public BlobStore blobStore(boolean serverSideEncryptionEnabled) { BlobStore store = null; if (serverSideEncryptionEnabled) { store = serverSideEncryptedBlobStore.get(); @@ -47,7 +50,7 @@ public BlobStore blobStore(IndexSettings indexSettings) { store = super.createBlobStore(serverSideEncryptedBlobStore, true); } } else { - store = super.blobStore(indexSettings); + store = super.blobStore(false); } return store; } @@ -55,20 +58,31 @@ public BlobStore blobStore(IndexSettings indexSettings) { /** * */ - protected BlobStore initBlobStore(boolean serverSideEncryption) { + protected BlobStore initBlobStore(boolean serverSideEncryptionEnabled) { if (lifecycle.started() == false) { throw new RepositoryException(metadata.name(), "repository is not in started state" + lifecycle.state()); } try { - if (serverSideEncryption) { + if (serverSideEncryptionEnabled) { return repository.createServerSideEncryptedBlobStore(); } else { return repository.createClientSideEncryptedBlobStore(); } - } catch (RepositoryException e) { + } catch (RepositoryException e) { throw e; } catch (Exception e) { throw new RepositoryException(metadata.name(), "cannot create blob store", e); } } + + public void close() { + super.close(); + try { + if (serverSideEncryptedBlobStore.get() != null) { + serverSideEncryptedBlobStore.get().close(); + } + } catch (Exception t) { + logger.warn("cannot close blob store", t); + } + } } diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 533597ecf3c98..73fe63b916dd7 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -240,7 +240,7 @@ private IndexService newIndexService(IndexModule module) throws IOException { indexSettings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 10), DefaultRemoteStoreSettings.INSTANCE, - indexSettings + indexSettings.isServerSideEncryptionEnabled() ); } return new InternalTranslogFactory(); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java index 95b5d4456baf6..b8ae85a5af2b3 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java @@ -71,6 +71,7 @@ public void testNewDirectory() throws IOException { BlobStoreRepository repository = mock(BlobStoreRepository.class); BlobStore blobStore = mock(BlobStore.class); BlobContainer blobContainer = mock(BlobContainer.class); + when(repository.blobStore(false)).thenReturn(blobStore); when(repository.blobStore()).thenReturn(blobStore); when(repository.basePath()).thenReturn(new BlobPath().add("base_path")); when(blobStore.blobContainer(any())).thenReturn(blobContainer); @@ -117,7 +118,7 @@ public void testNewDirectoryRepositoryDoesNotExist() { when(repositoriesService.repository("remote_store_repository")).thenThrow(new RepositoryMissingException("Missing")); - assertThrows(IllegalArgumentException.class, () -> remoteSegmentStoreDirectoryFactory.newDirectory(indexSettings, shardPath)); + assertThrows(RepositoryMissingException.class, () -> remoteSegmentStoreDirectoryFactory.newDirectory(indexSettings, shardPath)); } } diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java index 6c89cf2adf988..37fb222d0656d 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java @@ -151,7 +151,8 @@ protected RemoteFsTranslog createTranslogInstance( primaryMode::get, new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, - TranslogOperationHelper.DEFAULT + TranslogOperationHelper.DEFAULT, + false ); } @@ -622,7 +623,8 @@ public void testExtraGenToKeep() throws Exception { new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, - channelFactory + channelFactory, + false ) ) { addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index 237d99b586467..e609ca0bb92a1 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -203,7 +203,7 @@ protected RemoteFsTranslog createTranslogInstance( DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, null, - null + false ); } @@ -478,7 +478,8 @@ public void testExtraGenToKeep() throws Exception { new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, - null + null, + false ) ) { addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); @@ -1524,7 +1525,8 @@ public void testTranslogWriterCanFlushInAddOrReadCall() throws IOException { new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, - channelFactory + channelFactory, + false ) ) { TranslogWriter writer = translog.getCurrent(); @@ -1631,7 +1633,8 @@ public void force(boolean metaData) throws IOException { new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, - channelFactory + channelFactory, + false ) ) { TranslogWriter writer = translog.getCurrent(); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 270eb4f9882b4..a0df8ecd4bbef 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -743,7 +743,7 @@ protected IndexShard newShard( settings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 20), DefaultRemoteStoreSettings.INSTANCE, - settings + settings.isServerSideEncryptionEnabled() ); } return new InternalTranslogFactory(); From 8439c4ecbb1b7f78f0de2d287740e158e1db4d97 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Tue, 14 Oct 2025 20:17:33 +0530 Subject: [PATCH 04/10] Fix sse enabled property name. Also snapshot restore Signed-off-by: Pranit Kumar --- .../metadata/MetadataCreateIndexService.java | 9 +++++---- .../remotestore/RemoteStoreNodeAttribute.java | 15 +++++++++------ .../blobstore/BlobStoreProviderFactory.java | 2 +- .../org/opensearch/snapshots/RestoreService.java | 3 ++- 4 files changed, 17 insertions(+), 12 deletions(-) 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 09ecb3af613de..d17be974871b8 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1056,7 +1056,7 @@ static Settings aggregateIndexSettings( indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings, combinedTemplateSettings, clusterSettings); - updateRemoteStoreSettings(indexSettingsBuilder, currentState, clusterSettings, settings, request.index()); + updateRemoteStoreSettings(indexSettingsBuilder, currentState, clusterSettings, settings, request.index(), false); if (sourceMetadata != null) { assert request.resizeType() != null; @@ -1162,7 +1162,8 @@ public static void updateRemoteStoreSettings( ClusterState clusterState, ClusterSettings clusterSettings, Settings nodeSettings, - String indexName + String indexName, + boolean isRestoreFromSnapshot ) { if ((isRemoteDataAttributePresent(nodeSettings) && clusterSettings.get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(RemoteStoreNodeService.CompatibilityMode.STRICT)) @@ -1178,8 +1179,8 @@ public static void updateRemoteStoreSettings( if (remoteNode.isPresent()) { - if (RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes()) - && indexName.startsWith("sse-idx-1")) { + if (!isRestoreFromSnapshot + && RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes())) { settingsBuilder.put(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, true); } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index bb824c736a927..603047aa23ee8 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -44,6 +44,7 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.segment.repository"; public static final String REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.translog.repository"; public static final String REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.routing_table.repository"; + public static final String REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY = "server_side_encryption_enabled"; public static final List REMOTE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEYS = REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX.stream() .map(prefix -> prefix + ".state.repository") @@ -68,6 +69,7 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX = "remote_store.repository.%s.settings."; public static final String REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s.type"; + public static final String REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s.server_side_encryption_enabled"; public static final String REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s." + CryptoMetadata.CRYPTO_METADATA_KEY; public static final String REPOSITORY_CRYPTO_SETTINGS_PREFIX = REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT + "." @@ -249,9 +251,9 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { return false; } - public static boolean isRemoteStoreMetadata(Settings settings) { + public static boolean isServerSideEncryptionEnabled(Settings settings) { for (String metadataKey : settings.keySet()) { - if (metadataKey.equals("sse_enabled")) { + if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY)) { return true; } } @@ -289,10 +291,6 @@ public static boolean isRemoteClusterStateConfigured(Settings settings) { return false; } - public static boolean isRemoteStoreServerSideEncryptionEnabled(Map repos) { - return true; - } - public static String getRemoteStoreSegmentRepo(Settings settings) { for (String prefix : REMOTE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEYS) { if (settings.get(Node.NODE_ATTRIBUTES.getKey() + prefix) != null) { @@ -360,6 +358,11 @@ private static boolean remoteDataAttributesPresent(Map nodeAttrs return true; } + public static boolean isRemoteStoreServerSideEncryptionEnabled(Map repos) { + String attributeValue = repos.get(REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT); + return "true".equalsIgnoreCase(attributeValue); + } + public static String getClusterStateRepoName(Map repos) { return getValueFromAnyKey(repos, REMOTE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEYS); } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java index 409aad44d03e3..9eade1ade2d70 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java @@ -32,7 +32,7 @@ public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetada } public BlobStoreProvider getBlobStoreProvider() { - if (RemoteStoreNodeAttribute.isRemoteStoreMetadata(metadata.settings())) { + if (RemoteStoreNodeAttribute.isServerSideEncryptionEnabled(metadata.settings())) { if (serverSideEncryptionEnabledBlobStoreProvider.get() == null) { ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider = new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 3a333abe5b59f..e590bdff6918d 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -722,7 +722,8 @@ private Settings getOverrideSettingsInternal() { clusterService.state(), clusterSettings, clusterService.getSettings(), - String.join(",", request.indices()) + String.join(",", request.indices()), + true ); return settingsBuilder.build(); } From 01b6ca2af545c1f2333675523d5c09dc3f4107f9 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Tue, 14 Oct 2025 21:30:13 +0530 Subject: [PATCH 05/10] Fix public methods Signed-off-by: Pranit Kumar --- .../org/opensearch/index/shard/IndexShard.java | 17 +++++++++++++++++ .../RemoteSegmentStoreDirectoryFactory.java | 10 ++++++++++ 2 files changed, 27 insertions(+) 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 02e9ef196e35f..767f8a115b19e 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -5318,6 +5318,23 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { ); } + public void syncTranslogFilesFromGivenRemoteTranslog( + Repository repository, + ShardId shardId, + RemoteStorePathStrategy remoteStorePathStrategy, + boolean isTranslogMetadataEnabled, + long timestamp + ) throws IOException { + this.syncTranslogFilesFromGivenRemoteTranslog( + repository, + shardId, + remoteStorePathStrategy, + isTranslogMetadataEnabled, + timestamp, + false + ); + } + public void syncTranslogFilesFromGivenRemoteTranslog( Repository repository, ShardId shardId, diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 4704ed88e2cb7..83c7ea285d6c5 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -75,6 +75,16 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, null, false); } + public Directory newDirectory( + String repositoryName, + String indexUUID, + ShardId shardId, + RemoteStorePathStrategy pathStrategy, + String indexFixedPrefix + ) throws IOException { + return newDirectory(repositoryName, indexUUID, shardId, pathStrategy, indexFixedPrefix, false); + } + public Directory newDirectory( String repositoryName, String indexUUID, From 7b1273c900b5fcb3be526fe5f07401f7ea8b3d23 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Tue, 14 Oct 2025 21:34:22 +0530 Subject: [PATCH 06/10] Add ChangeLog file Signed-off-by: Pranit Kumar --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3cf0e78220f4a..b67c4f7b64ae0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - 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)) ### Changed From 0cb3731a8584ebdc9d45eb72b1f4a70a6573b743 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Thu, 16 Oct 2025 09:02:50 +0530 Subject: [PATCH 07/10] Add test cases Signed-off-by: Pranit Kumar --- .../repositories/s3/S3Repository.java | 2 +- .../repositories/s3/S3RepositoryTests.java | 34 +++ .../cluster/metadata/IndexMetadata.java | 1 + .../metadata/MetadataCreateIndexService.java | 6 +- .../common/settings/ClusterSettings.java | 2 + .../indices/RemoteStoreSettings.java | 31 ++ .../remotestore/RemoteStoreNodeAttribute.java | 15 +- .../blobstore/BlobStoreProviderFactory.java | 26 +- .../MetadataCreateIndexServiceTests.java | 100 ++++++ .../RemoteFsTimestampAwareTranslogTests.java | 5 +- .../index/translog/RemoteFsTranslogTests.java | 28 +- .../BlobStoreProviderFactoryTests.java | 222 ++++++++++++++ ...cryptionEnabledBlobStoreProviderTests.java | 286 ++++++++++++++++++ 13 files changed, 727 insertions(+), 31 deletions(-) create mode 100644 server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java create mode 100644 server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java 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 6d543a270785f..6bbad99a45d7d 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 @@ -531,7 +531,7 @@ protected S3BlobStore createClientSideEncryptedBlobStore() { null, false, null, - "" + null ); } 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..1c29f8178dff3 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,39 @@ public void testValidateHttpLClientType_Invalid_Values() { } } + public void testCreateClientSideEncryptedBlobStore() { + final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", Settings.EMPTY); + try (S3Repository s3Repo = createS3Repo(metadata)) { + // Don't expect any Exception + S3BlobStore blobStore = s3Repo.createClientSideEncryptedBlobStore(); + assertNotNull(blobStore); + assertNull(blobStore.serverSideEncryptionKmsKey()); + assertEquals(blobStore.serverSideEncryptionType(), ServerSideEncryption.AES256.toString()); + assertNull(blobStore.expectedBucketOwner()); + assertFalse(blobStore.serverSideEncryptionBucketKey()); + } + } + + public void testCreateServerSideEncryptedBlobStore() { + Settings settings = Settings.builder() + .put("server_side_encryption_type", ServerSideEncryption.AWS_KMS.toString()) + .put("server_side_encryption_kms_key_id", "kms-key-id") + .put("server_side_encryption_bucket_key_enabled", true) + .put("server_side_encryption_encryption_context", "enc-ctx") + .put("expected_bucket_owner", "123456789012") + .build(); + + final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", settings); + try (S3Repository s3Repo = createS3Repo(metadata)) { + // Don't expect any Exception + S3BlobStore blobStore = s3Repo.createServerSideEncryptedBlobStore(); + assertNotNull(blobStore); + assertEquals("kms-key-id", blobStore.serverSideEncryptionKmsKey()); + assertEquals("123456789012", blobStore.expectedBucketOwner()); + assertTrue(blobStore.serverSideEncryptionBucketKey()); + } + } + private S3Repository createS3Repo(RepositoryMetadata metadata) { return new S3Repository( metadata, 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 9b8e6c83fa272..1925322e4cbbc 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -1025,6 +1025,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 SERVER_SIDE_ENCRYPTION_ENABLED = "server_side_encryption_enabled"; 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 d17be974871b8..b2ec6f3228019 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1178,14 +1178,14 @@ public static void updateRemoteStoreSettings( .findFirst(); if (remoteNode.isPresent()) { - + segmentRepo = RemoteStoreNodeAttribute.getSegmentRepoName(remoteNode.get().getAttributes()); if (!isRestoreFromSnapshot - && RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes())) { + && RemoteStoreSettings.isServerSideEncryptionRepoEnabled(clusterState.nodes().getMinNodeVersion()) + && RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes(), segmentRepo)) { settingsBuilder.put(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, true); } translogRepo = RemoteStoreNodeAttribute.getTranslogRepoName(remoteNode.get().getAttributes()); - segmentRepo = RemoteStoreNodeAttribute.getSegmentRepoName(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 572d8ab2914c7..bf5ba8515ed1e 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -810,6 +810,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_REPO_ENABLED, // Snapshot related Settings BlobStoreRepository.SNAPSHOT_SHARD_PATH_PREFIX_SETTING, diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 1f09af234ae30..02106e5512c84 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -8,6 +8,7 @@ package org.opensearch.indices; +import org.opensearch.Version; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.ClusterSettings; @@ -184,6 +185,16 @@ public class RemoteStoreSettings { Property.Final ); + /** + * Controls the ServerSideEncryption Settings. + */ + public static final Setting CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED = Setting.boolSetting( + "cluster.server_side_encryption.repo.enabled", + true, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private volatile TimeValue clusterRemoteTranslogBufferInterval; private volatile int minRemoteSegmentMetadataFiles; private volatile TimeValue clusterRemoteTranslogTransferTimeout; @@ -191,6 +202,7 @@ public class RemoteStoreSettings { private volatile RemoteStoreEnums.PathType pathType; private volatile RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm; private volatile int maxRemoteTranslogReaders; + private static volatile boolean isClusterServerSideEncryptionRepoEnabled; private volatile boolean isTranslogMetadataEnabled; private static volatile boolean isPinnedTimestampsEnabled; private static volatile TimeValue pinnedTimestampsSchedulerInterval; @@ -235,6 +247,12 @@ public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { this::setClusterRemoteSegmentTransferTimeout ); + isClusterServerSideEncryptionRepoEnabled = CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.get(settings); + clusterSettings.addSettingsUpdateConsumer( + CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED, + RemoteStoreSettings::setClusterServerSideEncryptionRepoEnabled + ); + pinnedTimestampsSchedulerInterval = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_SCHEDULER_INTERVAL.get(settings); pinnedTimestampsLookbackInterval = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_LOOKBACK_INTERVAL.get(settings); isPinnedTimestampsEnabled = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.get(settings); @@ -309,6 +327,14 @@ private void setMaxRemoteTranslogReaders(int maxRemoteTranslogReaders) { this.maxRemoteTranslogReaders = maxRemoteTranslogReaders; } + public static boolean isClusterServerSideEncryptionRepoEnabled() { + return isClusterServerSideEncryptionRepoEnabled; + } + + private static void setClusterServerSideEncryptionRepoEnabled(boolean clusterServerSideEncryptionRepoEnabled) { + isClusterServerSideEncryptionRepoEnabled = clusterServerSideEncryptionRepoEnabled; + } + public static TimeValue getPinnedTimestampsSchedulerInterval() { return pinnedTimestampsSchedulerInterval; } @@ -333,4 +359,9 @@ public String getTranslogPathFixedPrefix() { public String getSegmentsPathFixedPrefix() { return segmentsPathFixedPrefix; } + + public static boolean isServerSideEncryptionRepoEnabled(Version minNodeVersion) { + return Version.V_3_1_0.compareTo(minNodeVersion) <= 0 + && isClusterServerSideEncryptionRepoEnabled(); + } } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 603047aa23ee8..70a6da1a7b810 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -69,7 +69,9 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX = "remote_store.repository.%s.settings."; public static final String REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s.type"; - public static final String REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s.server_side_encryption_enabled"; + public static final String REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT = + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX + ".server_side_encryption_enabled"; + public static final String REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s." + CryptoMetadata.CRYPTO_METADATA_KEY; public static final String REPOSITORY_CRYPTO_SETTINGS_PREFIX = REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT + "." @@ -253,7 +255,8 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { public static boolean isServerSideEncryptionEnabled(Settings settings) { for (String metadataKey : settings.keySet()) { - if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY)) { + if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY) + && settings.getAsBoolean(metadataKey, false)) { return true; } } @@ -358,8 +361,12 @@ private static boolean remoteDataAttributesPresent(Map nodeAttrs return true; } - public static boolean isRemoteStoreServerSideEncryptionEnabled(Map repos) { - String attributeValue = repos.get(REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT); + public static boolean isRemoteStoreServerSideEncryptionEnabled( + Map nodeAttributes, + String repoName) { + + String attributeKey = String.format(Locale.getDefault(), REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, repoName); + String attributeValue = nodeAttributes.get(attributeKey); return "true".equalsIgnoreCase(attributeValue); } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java index 9eade1ade2d70..41a13e596c24e 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java @@ -32,19 +32,21 @@ public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetada } public BlobStoreProvider getBlobStoreProvider() { - if (RemoteStoreNodeAttribute.isServerSideEncryptionEnabled(metadata.settings())) { - if (serverSideEncryptionEnabledBlobStoreProvider.get() == null) { - ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider = - new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); - this.serverSideEncryptionEnabledBlobStoreProvider.set(serverSideEncryptionEnabledBlobStoreProvider); + synchronized (lock) { + if (RemoteStoreNodeAttribute.isServerSideEncryptionEnabled(metadata.settings())) { + if (serverSideEncryptionEnabledBlobStoreProvider.get() == null) { + ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider = + new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); + this.serverSideEncryptionEnabledBlobStoreProvider.set(serverSideEncryptionEnabledBlobStoreProvider); + } + return serverSideEncryptionEnabledBlobStoreProvider.get(); + } else { + if (blobStoreProvider.get() == null) { + BlobStoreProvider blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); + this.blobStoreProvider.set(blobStoreProvider); + } + return blobStoreProvider.get(); } - return serverSideEncryptionEnabledBlobStoreProvider.get(); - } else { - if (blobStoreProvider.get() == null) { - BlobStoreProvider blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); - this.blobStoreProvider.set(blobStoreProvider); - } - return blobStoreProvider.get(); } } } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 785ba35afaf4d..69889b8b7b78f 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -93,6 +93,7 @@ import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.indices.SystemIndices; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -174,6 +175,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; @@ -2615,6 +2617,103 @@ public void testIndexTotalPrimaryShardsPerNodeSettingValidationWithoutRemoteStor ); } + public void testUpdateRemoteStoreSettings() { + Map attributes = getNodeAttributes(); + DiscoveryNode remoteNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attributes, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(remoteNode).build()) + .build(); + + Settings settings = Settings.builder() + .put("node.attr.remote_store.segment.repository", "my-segment-repo-1") + .build(); + final Settings.Builder requestSettings = Settings.builder() + .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); + + Settings clusterSettingsSetting = Settings.builder() + .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + .build(); + clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + new RemoteStoreSettings( + clusterSettingsSetting, + clusterSettings + ); + + MetadataCreateIndexService.updateRemoteStoreSettings( + requestSettings, + clusterState, + clusterSettings, + settings, + "test-index", + false); + + assertTrue(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); + } + + public void testUpdateRemoteStoreSettings_For_Snapshot_restore() { + Map attributes = getNodeAttributes(); + DiscoveryNode remoteNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attributes, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(remoteNode).build()) + .build(); + + Settings settings = Settings.builder() + .put("node.attr.remote_store.segment.repository", "my-segment-repo-1") + .build(); + final Settings.Builder requestSettings = Settings.builder() + .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); + + Settings clusterSettingsSetting = Settings.builder() + .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + .build(); + clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + new RemoteStoreSettings( + clusterSettingsSetting, + clusterSettings + ); + + MetadataCreateIndexService.updateRemoteStoreSettings( + requestSettings, + clusterState, + clusterSettings, + settings, + "test-index", + true); + + assertFalse(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); + } + + private static Map getNodeAttributes() { + String segmentRepositoryName = "my-segment-repo-1"; + String serverSideEncryptionAttribute = String.format(Locale.getDefault(), + RemoteStoreNodeAttribute.REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, segmentRepositoryName); + Map attributes = new HashMap<>(); + + attributes.put(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-cluster-rep-1"); + attributes.put(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, segmentRepositoryName); + attributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); + attributes.put(serverSideEncryptionAttribute, "true"); + return attributes; + } + public void testIndexTotalPrimaryShardsPerNodeSettingValidationWithDefaultValue() { // Test case with default value (-1) without remote store (should succeed) Settings settings = Settings.builder().build(); @@ -2704,6 +2803,7 @@ private DiscoveryNode getRemoteNode() { attributes.put(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-cluster-rep-1"); attributes.put(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-segment-repo-1"); attributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); + attributes.put(REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, "my-translog-repo-1"); return new DiscoveryNode( UUIDs.base64UUID(), buildNewFakeTransportAddress(), diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java index 37fb222d0656d..5ab26084dcfd2 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java @@ -137,7 +137,8 @@ public void setUp() throws Exception { protected RemoteFsTranslog createTranslogInstance( TranslogConfig translogConfig, String translogUUID, - TranslogDeletionPolicy deletionPolicy + TranslogDeletionPolicy deletionPolicy, + boolean isServerSideEncryptionEnabled ) throws IOException { return new RemoteFsTimestampAwareTranslog( translogConfig, @@ -152,7 +153,7 @@ protected RemoteFsTranslog createTranslogInstance( new RemoteTranslogTransferTracker(shardId, 10), DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, - false + isServerSideEncryptionEnabled ); } diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index e609ca0bb92a1..b38946114cecb 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -171,23 +171,29 @@ public void tearDown() throws Exception { protected RemoteFsTranslog create(Path path) throws IOException { final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); - return create(path, createRepository(), translogUUID, 0); + return create(path, createRepository(), translogUUID, 0, false); } - private RemoteFsTranslog create(Path path, BlobStoreRepository repository, String translogUUID, int extraGenToKeep) throws IOException { + private RemoteFsTranslog create( + Path path, + BlobStoreRepository repository, + String translogUUID, + int extraGenToKeep, + boolean isServerSideEncryptionEnabled) throws IOException { this.repository = repository; globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final TranslogConfig translogConfig = getTranslogConfig(path, extraGenToKeep); final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings()); threadPool = new TestThreadPool(getClass().getName()); blobStoreTransferService = new BlobStoreTransferService(repository.blobStore(), threadPool); - return createTranslogInstance(translogConfig, translogUUID, deletionPolicy); + return createTranslogInstance(translogConfig, translogUUID, deletionPolicy, isServerSideEncryptionEnabled); } protected RemoteFsTranslog createTranslogInstance( TranslogConfig translogConfig, String translogUUID, - TranslogDeletionPolicy deletionPolicy + TranslogDeletionPolicy deletionPolicy, + boolean isServerSideEncryptionEnabled ) throws IOException { return new RemoteFsTranslog( translogConfig, @@ -203,12 +209,16 @@ protected RemoteFsTranslog createTranslogInstance( DefaultRemoteStoreSettings.INSTANCE, TranslogOperationHelper.DEFAULT, null, - false + isServerSideEncryptionEnabled ); } - private RemoteFsTranslog create(Path path, BlobStoreRepository repository, String translogUUID) throws IOException { - return create(path, repository, translogUUID, 0); + private RemoteFsTranslog create( + Path path, + BlobStoreRepository repository, + String translogUUID, + boolean isServerSideEncryptionEnabled) throws IOException { + return create(path, repository, translogUUID, 0,isServerSideEncryptionEnabled); } private TranslogConfig getTranslogConfig(final Path path) { @@ -556,7 +566,7 @@ public void testReadLocationDownload() throws IOException { } // Creating RemoteFsTranslog with the same location - RemoteFsTranslog newTranslog = create(translogDir, repository, translogUUID); + RemoteFsTranslog newTranslog = create(translogDir, repository, translogUUID, false); i = 0; for (Translog.Operation op : ops) { assertEquals(op, newTranslog.readOperation(locs.get(i++))); @@ -827,7 +837,7 @@ public void testMetadataFileDeletion() throws Exception { long newPrimaryTerm = primaryTerm.incrementAndGet(); // Creating RemoteFsTranslog with the same location - Translog newTranslog = create(translogDir, repository, translogUUID); + Translog newTranslog = create(translogDir, repository, translogUUID, false); int newPrimaryTermDocs = randomIntBetween(5, 10); for (int i = totalDocs + 1; i <= totalDocs + newPrimaryTermDocs; i++) { addToTranslogAndListAndUpload(newTranslog, ops, new Translog.Index(String.valueOf(i), i, primaryTerm.get(), new byte[] { 1 })); diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java new file mode 100644 index 0000000000000..ce5d4e1b9391b --- /dev/null +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java @@ -0,0 +1,222 @@ +/* + * 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.repositories.blobstore; + + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.common.settings.Settings; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; + +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.atLeastOnce; + +import org.opensearch.test.OpenSearchTestCase; + +public class BlobStoreProviderFactoryTests extends OpenSearchTestCase { + + @Mock + private BlobStoreRepository mockRepository; + + @Mock + private RepositoryMetadata mockMetadata; + + @Mock + private Lifecycle mockLifecycle; + + private Object lock; + private BlobStoreProviderFactory factory; + + @Before + public void setUp() throws Exception { + super.setUp(); + MockitoAnnotations.initMocks(this); + lock = new Object(); + } + + @Test + public void testConstructor() { + // Test successful construction + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + assertNotNull(factory); + } + + @Test + public void testGetBlobStoreProviderWithServerSideEncryptionEnabled() { + // Setup + Settings settings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) + .build(); + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute + BlobStoreProvider provider = factory.getBlobStoreProvider(); + + // Verify + assertNotNull(provider); + assertTrue(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); + verify(mockMetadata, atLeastOnce()).settings(); + } + + @Test + public void testGetBlobStoreProviderWithServerSideEncryptionDisabled() { + // Setup + Settings settings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) + .build(); + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute + BlobStoreProvider provider = factory.getBlobStoreProvider(); + + // Verify + assertNotNull(provider); + System.out.println("provider.getClass().getName() = " + provider.getClass().getName()); + assertFalse(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); + assertTrue(provider instanceof BlobStoreProvider); + verify(mockMetadata, atLeastOnce()).settings(); + } + + @Test + public void testGetBlobStoreProviderWithDefaultSettings() { + // Setup - empty settings (default behavior) + Settings settings = Settings.EMPTY; + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute + BlobStoreProvider provider = factory.getBlobStoreProvider(); + + // Verify + assertNotNull(provider); + assertFalse(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); + assertTrue(provider instanceof BlobStoreProvider); + } + + @Test + public void testGetBlobStoreProviderSingletonBehaviorWithEncryption() { + // Setup + Settings settings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) + .build(); + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute multiple calls + BlobStoreProvider provider1 = factory.getBlobStoreProvider(); + BlobStoreProvider provider2 = factory.getBlobStoreProvider(); + + // Verify same instance is returned (singleton behavior) + assertSame(provider1, provider2); + assertTrue(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); + } + + @Test + public void testGetBlobStoreProviderSingletonBehaviorWithoutEncryption() { + // Setup + Settings settings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) + .build(); + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute multiple calls + BlobStoreProvider provider1 = factory.getBlobStoreProvider(); + BlobStoreProvider provider2 = factory.getBlobStoreProvider(); + + // Verify same instance is returned (singleton behavior) + assertSame(provider1, provider2); + assertFalse(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); + } + + @Test + public void testGetBlobStoreProviderThreadSafety() throws InterruptedException { + // Setup + Settings settings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) + .build(); + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Test concurrent access + final BlobStoreProvider[] providers = new BlobStoreProvider[2]; + Thread thread1 = new Thread(() -> providers[0] = factory.getBlobStoreProvider()); + Thread thread2 = new Thread(() -> providers[1] = factory.getBlobStoreProvider()); + + thread1.start(); + thread2.start(); + + thread1.join(); + thread2.join(); + + // Verify both threads get the same instance + assertNotNull(providers[0]); + assertNotNull(providers[1]); + assertSame(providers[0], providers[1]); + } + + @Test + public void testGetBlobStoreProviderParameterPassing() { + // Setup + Settings settings = Settings.EMPTY; + when(mockMetadata.settings()).thenReturn(settings); + + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + // Execute + BlobStoreProvider provider = factory.getBlobStoreProvider(); + + // Verify that the provider was created with correct parameters + // This test ensures that the factory passes the correct constructor parameters + assertNotNull(provider); + + // Additional verification could be done if BlobStoreProvider had getter methods + // or if we could verify the constructor calls through mocking + } + + @Test + public void testFactoryStateConsistency() { + // Setup + Settings encryptedSettings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) + .build(); + Settings nonEncryptedSettings = Settings.builder() + .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) + .build(); + + // Test that factory maintains consistent state based on initial metadata + when(mockMetadata.settings()).thenReturn(encryptedSettings); + factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); + + BlobStoreProvider encryptedProvider = factory.getBlobStoreProvider(); + assertTrue(encryptedProvider instanceof ServerSideEncryptionEnabledBlobStoreProvider); + + // Even if we change the mock to return different settings, + // the factory should maintain its initial behavior + when(mockMetadata.settings()).thenReturn(nonEncryptedSettings); + BlobStoreProvider sameProvider = factory.getBlobStoreProvider(); + + assertNotSame(encryptedProvider, sameProvider); + assertTrue(sameProvider != null); + } + +} diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java new file mode 100644 index 0000000000000..8522970391e63 --- /dev/null +++ b/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java @@ -0,0 +1,286 @@ +/* + * 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.repositories.blobstore; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.repositories.RepositoryException; + +import static org.mockito.Mockito.*; + +import org.opensearch.test.OpenSearchTestCase; + +public class ServerSideEncryptionEnabledBlobStoreProviderTests extends OpenSearchTestCase { + + @Mock + private BlobStoreRepository mockRepository; + + @Mock + private RepositoryMetadata mockMetadata; + + @Mock + private Lifecycle mockLifecycle; + + @Mock + private BlobStore mockServerSideEncryptedBlobStore; + + @Mock + private BlobStore mockClientSideEncryptedBlobStore; + + @Mock + private BlobStore mockRegularBlobStore; + + private Object lock; + private ServerSideEncryptionEnabledBlobStoreProvider provider; + + @Before + public void setUp() throws Exception { + super.setUp(); + MockitoAnnotations.openMocks(this); + lock = new Object(); + when(mockMetadata.name()).thenReturn("test-repository"); + provider = new ServerSideEncryptionEnabledBlobStoreProvider( + mockRepository, + mockMetadata, + mockLifecycle, + lock + ); + } + + @Test + public void testConstructor() { + assertNotNull(provider); + // Verify that the provider extends BlobStoreProvider + assertTrue(provider instanceof BlobStoreProvider); + } + + @Test + public void testGetBlobStoreWithServerSideEncryptionEnabled() { + // Setup: Mock the serverSideEncryptedBlobStore to return a value + // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + + // Initialize the server-side encrypted blob store + provider.blobStore(true); + + // Test + BlobStore result = provider.getBlobStore(true); + + // Verify + assertEquals(mockServerSideEncryptedBlobStore, result); + } + + @Test + public void testGetBlobStoreWithServerSideEncryptionDisabled() { + // Setup: Mock the regular blobStore + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockRegularBlobStore); + + // Initialize the regular blob store + provider.blobStore(false); + + // Test + BlobStore result = provider.getBlobStore(false); + + // Verify + assertEquals(mockRegularBlobStore, result); + } + + @Test + public void testBlobStoreWithServerSideEncryptionEnabledFirstTime() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + + // Test + BlobStore result = provider.blobStore(true); + + // Verify + assertEquals(mockServerSideEncryptedBlobStore, result); + verify(mockRepository).createServerSideEncryptedBlobStore(); + } + + @Test + public void testBlobStoreWithServerSideEncryptionEnabledSubsequentCalls() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + + // First call + BlobStore firstResult = provider.blobStore(true); + + // Second call + BlobStore secondResult = provider.blobStore(true); + + // Verify + assertEquals(mockServerSideEncryptedBlobStore, firstResult); + assertEquals(mockServerSideEncryptedBlobStore, secondResult); + assertSame(firstResult, secondResult); + // Verify createServerSideEncryptedBlobStore is called only once + verify(mockRepository, times(1)).createServerSideEncryptedBlobStore(); + } + + @Test + public void testBlobStoreWithServerSideEncryptionDisabled() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); + + // Test + BlobStore result = provider.blobStore(false); + + // Verify + assertEquals(mockClientSideEncryptedBlobStore, result); + } + + @Test + public void testInitBlobStoreWithServerSideEncryptionEnabled() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + + // Test + BlobStore result = provider.initBlobStore(true); + + // Verify + assertEquals(mockServerSideEncryptedBlobStore, result); + verify(mockRepository).createServerSideEncryptedBlobStore(); + } + + @Test + public void testInitBlobStoreWithServerSideEncryptionDisabled() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); + + // Test + BlobStore result = provider.initBlobStore(false); + + // Verify + assertEquals(mockClientSideEncryptedBlobStore, result); + verify(mockRepository).createClientSideEncryptedBlobStore(); + } + + @Test(expected = RepositoryException.class) + public void testInitBlobStoreWhenLifecycleNotStarted() { + // Setup + when(mockLifecycle.started()).thenReturn(false); + when(mockLifecycle.state()).thenReturn(Lifecycle.State.STOPPED); + + // Test - should throw RepositoryException + provider.initBlobStore(true); + } + + @Test(expected = RepositoryException.class) + public void testInitBlobStoreWhenRepositoryThrowsRepositoryException() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + RepositoryException repositoryException = new RepositoryException("test-repo", "test error"); + when(mockRepository.createServerSideEncryptedBlobStore()).thenThrow(repositoryException); + + // Test - should re-throw RepositoryException + provider.initBlobStore(true); + } + + @Test(expected = RepositoryException.class) + public void testInitBlobStoreWhenRepositoryThrowsGenericException() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + RuntimeException genericException = new RuntimeException("generic error"); + when(mockRepository.createServerSideEncryptedBlobStore()).thenThrow(genericException); + + // Test - should wrap in RepositoryException + try { + provider.initBlobStore(true); + } catch (RepositoryException e) { + assertEquals("test-repository", e.repository()); + assertEquals("[test-repository] cannot create blob store", e.getMessage()); + assertEquals(genericException, e.getCause()); + throw e; + } + } + + @Test + public void testCloseWithServerSideEncryptedBlobStore() throws Exception { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + + // Initialize the server-side encrypted blob store + provider.blobStore(true); + + // Test + provider.close(); + + // Verify + verify(mockServerSideEncryptedBlobStore).close(); + } + + @Test + public void testCloseWithoutServerSideEncryptedBlobStore() throws Exception { + // Test - close without initializing server-side encrypted blob store + provider.close(); + + // Verify - no exception should be thrown and no close() called on null store + // This test passes if no exception is thrown + } + + @Test + public void testCloseWhenServerSideEncryptedBlobStoreThrowsException() throws Exception { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + doThrow(new RuntimeException("close error")).when(mockServerSideEncryptedBlobStore).close(); + + // Initialize the server-side encrypted blob store + provider.blobStore(true); + + // Test - should not throw exception even if blob store close() fails + provider.close(); + + // Verify + verify(mockServerSideEncryptedBlobStore).close(); + // Test passes if no exception is propagated + } + + @Test + public void testMixedUsageServerSideAndClientSide() { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); + when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); + + // Test - use both server-side and client-side encryption + BlobStore serverSideResult = provider.blobStore(true); + BlobStore clientSideResult = provider.blobStore(false); + + // Verify + assertEquals(mockServerSideEncryptedBlobStore, serverSideResult); + assertEquals(mockClientSideEncryptedBlobStore, clientSideResult); + assertNotSame(serverSideResult, clientSideResult); + + verify(mockRepository).createServerSideEncryptedBlobStore(); + verify(mockRepository).createClientSideEncryptedBlobStore(); + } + + @Test + public void testGetBlobStoreReturnsNullWhenNotInitialized() { + // Test - getBlobStore when server-side encrypted store is not initialized + BlobStore result = provider.getBlobStore(true); + + // Verify - should return null since SetOnce.get() returns null when not set + assertNull(result); + } +} From 37a09509d2d1b79bd8423cd171b74025a2a55cb4 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Thu, 16 Oct 2025 10:02:52 +0530 Subject: [PATCH 08/10] Fix spotless and forbiddenAPI issues Signed-off-by: Pranit Kumar --- .../indices/RemoteStoreSettings.java | 3 +- .../remotestore/RemoteStoreNodeAttribute.java | 7 +- .../blobstore/BlobStoreProvider.java | 3 +- .../MetadataCreateIndexServiceTests.java | 41 ++---- .../index/translog/RemoteFsTranslogTests.java | 12 +- .../BlobStoreProviderFactoryTests.java | 26 +--- .../blobstore/BlobStoreProviderTests.java | 126 ++++++++++++++++++ ...cryptionEnabledBlobStoreProviderTests.java | 44 ++---- 8 files changed, 165 insertions(+), 97 deletions(-) create mode 100644 server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 02106e5512c84..16320f66429e5 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -361,7 +361,6 @@ public String getSegmentsPathFixedPrefix() { } public static boolean isServerSideEncryptionRepoEnabled(Version minNodeVersion) { - return Version.V_3_1_0.compareTo(minNodeVersion) <= 0 - && isClusterServerSideEncryptionRepoEnabled(); + return Version.V_3_1_0.compareTo(minNodeVersion) <= 0 && isClusterServerSideEncryptionRepoEnabled(); } } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 70a6da1a7b810..0c010db22d026 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -255,8 +255,7 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { public static boolean isServerSideEncryptionEnabled(Settings settings) { for (String metadataKey : settings.keySet()) { - if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY) - && settings.getAsBoolean(metadataKey, false)) { + if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY) && settings.getAsBoolean(metadataKey, false)) { return true; } } @@ -361,9 +360,7 @@ private static boolean remoteDataAttributesPresent(Map nodeAttrs return true; } - public static boolean isRemoteStoreServerSideEncryptionEnabled( - Map nodeAttributes, - String repoName) { + public static boolean isRemoteStoreServerSideEncryptionEnabled(Map nodeAttributes, String repoName) { String attributeKey = String.format(Locale.getDefault(), REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, repoName); String attributeValue = nodeAttributes.get(attributeKey); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java index a22158c3c945f..714eee8745f4f 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java @@ -49,7 +49,7 @@ public BlobStore blobStore() { protected BlobStore createBlobStore(SetOnce blobStore, boolean serverSideEncryption) { // assertSnapshotOrGenericThread(); BlobStore store = blobStore.get(); - logger.info("1.store = " + store); + logger.debug("blob store fetched = " + store); if (store == null) { synchronized (lock) { store = blobStore.get(); @@ -62,7 +62,6 @@ protected BlobStore createBlobStore(SetOnce blobStore, boolean server } } } - logger.info("2.store = " + store); return store; } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 69889b8b7b78f..fbd172be9a15b 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -2630,9 +2630,7 @@ public void testUpdateRemoteStoreSettings() { .nodes(DiscoveryNodes.builder().add(remoteNode).build()) .build(); - Settings settings = Settings.builder() - .put("node.attr.remote_store.segment.repository", "my-segment-repo-1") - .build(); + Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); final Settings.Builder requestSettings = Settings.builder() .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); @@ -2643,18 +2641,9 @@ public void testUpdateRemoteStoreSettings() { .build(); clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - new RemoteStoreSettings( - clusterSettingsSetting, - clusterSettings - ); + new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); - MetadataCreateIndexService.updateRemoteStoreSettings( - requestSettings, - clusterState, - clusterSettings, - settings, - "test-index", - false); + MetadataCreateIndexService.updateRemoteStoreSettings(requestSettings, clusterState, clusterSettings, settings, "test-index", false); assertTrue(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); } @@ -2672,9 +2661,7 @@ public void testUpdateRemoteStoreSettings_For_Snapshot_restore() { .nodes(DiscoveryNodes.builder().add(remoteNode).build()) .build(); - Settings settings = Settings.builder() - .put("node.attr.remote_store.segment.repository", "my-segment-repo-1") - .build(); + Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); final Settings.Builder requestSettings = Settings.builder() .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); @@ -2685,26 +2672,20 @@ public void testUpdateRemoteStoreSettings_For_Snapshot_restore() { .build(); clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - new RemoteStoreSettings( - clusterSettingsSetting, - clusterSettings - ); + new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); - MetadataCreateIndexService.updateRemoteStoreSettings( - requestSettings, - clusterState, - clusterSettings, - settings, - "test-index", - true); + MetadataCreateIndexService.updateRemoteStoreSettings(requestSettings, clusterState, clusterSettings, settings, "test-index", true); assertFalse(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); } private static Map getNodeAttributes() { String segmentRepositoryName = "my-segment-repo-1"; - String serverSideEncryptionAttribute = String.format(Locale.getDefault(), - RemoteStoreNodeAttribute.REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, segmentRepositoryName); + String serverSideEncryptionAttribute = String.format( + Locale.getDefault(), + RemoteStoreNodeAttribute.REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, + segmentRepositoryName + ); Map attributes = new HashMap<>(); attributes.put(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-cluster-rep-1"); diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index b38946114cecb..7b20c7e22f2f4 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -179,7 +179,8 @@ private RemoteFsTranslog create( BlobStoreRepository repository, String translogUUID, int extraGenToKeep, - boolean isServerSideEncryptionEnabled) throws IOException { + boolean isServerSideEncryptionEnabled + ) throws IOException { this.repository = repository; globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final TranslogConfig translogConfig = getTranslogConfig(path, extraGenToKeep); @@ -213,12 +214,9 @@ protected RemoteFsTranslog createTranslogInstance( ); } - private RemoteFsTranslog create( - Path path, - BlobStoreRepository repository, - String translogUUID, - boolean isServerSideEncryptionEnabled) throws IOException { - return create(path, repository, translogUUID, 0,isServerSideEncryptionEnabled); + private RemoteFsTranslog create(Path path, BlobStoreRepository repository, String translogUUID, boolean isServerSideEncryptionEnabled) + throws IOException { + return create(path, repository, translogUUID, 0, isServerSideEncryptionEnabled); } private TranslogConfig getTranslogConfig(final Path path) { diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java index ce5d4e1b9391b..e720d06b92b06 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java @@ -8,21 +8,19 @@ package org.opensearch.repositories.blobstore; - -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.common.settings.Settings; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.atLeastOnce; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; -import org.opensearch.test.OpenSearchTestCase; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class BlobStoreProviderFactoryTests extends OpenSearchTestCase { @@ -45,14 +43,12 @@ public void setUp() throws Exception { lock = new Object(); } - @Test public void testConstructor() { // Test successful construction factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); assertNotNull(factory); } - @Test public void testGetBlobStoreProviderWithServerSideEncryptionEnabled() { // Setup Settings settings = Settings.builder() @@ -71,7 +67,6 @@ public void testGetBlobStoreProviderWithServerSideEncryptionEnabled() { verify(mockMetadata, atLeastOnce()).settings(); } - @Test public void testGetBlobStoreProviderWithServerSideEncryptionDisabled() { // Setup Settings settings = Settings.builder() @@ -86,13 +81,11 @@ public void testGetBlobStoreProviderWithServerSideEncryptionDisabled() { // Verify assertNotNull(provider); - System.out.println("provider.getClass().getName() = " + provider.getClass().getName()); assertFalse(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); assertTrue(provider instanceof BlobStoreProvider); verify(mockMetadata, atLeastOnce()).settings(); } - @Test public void testGetBlobStoreProviderWithDefaultSettings() { // Setup - empty settings (default behavior) Settings settings = Settings.EMPTY; @@ -109,7 +102,6 @@ public void testGetBlobStoreProviderWithDefaultSettings() { assertTrue(provider instanceof BlobStoreProvider); } - @Test public void testGetBlobStoreProviderSingletonBehaviorWithEncryption() { // Setup Settings settings = Settings.builder() @@ -128,7 +120,6 @@ public void testGetBlobStoreProviderSingletonBehaviorWithEncryption() { assertTrue(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); } - @Test public void testGetBlobStoreProviderSingletonBehaviorWithoutEncryption() { // Setup Settings settings = Settings.builder() @@ -147,7 +138,6 @@ public void testGetBlobStoreProviderSingletonBehaviorWithoutEncryption() { assertFalse(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); } - @Test public void testGetBlobStoreProviderThreadSafety() throws InterruptedException { // Setup Settings settings = Settings.builder() @@ -174,7 +164,6 @@ public void testGetBlobStoreProviderThreadSafety() throws InterruptedException { assertSame(providers[0], providers[1]); } - @Test public void testGetBlobStoreProviderParameterPassing() { // Setup Settings settings = Settings.EMPTY; @@ -193,7 +182,6 @@ public void testGetBlobStoreProviderParameterPassing() { // or if we could verify the constructor calls through mocking } - @Test public void testFactoryStateConsistency() { // Setup Settings encryptedSettings = Settings.builder() diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java new file mode 100644 index 0000000000000..9bf3c0d3a4b8c --- /dev/null +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java @@ -0,0 +1,126 @@ +/* + * 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.repositories.blobstore; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.repositories.RepositoryException; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class BlobStoreProviderTests extends OpenSearchTestCase { + @Mock + private BlobStoreRepository mockRepository; + + @Mock + private RepositoryMetadata mockMetadata; + + @Mock + private Lifecycle mockLifecycle; + + @Mock + private BlobStore mockBlobStore; + + @Mock + private BlobStore mockRegularBlobStore; + + private Object lock; + private BlobStoreProvider provider; + + @Before + public void setUp() throws Exception { + super.setUp(); + MockitoAnnotations.openMocks(this); + lock = new Object(); + when(mockMetadata.name()).thenReturn("test-repository"); + provider = new BlobStoreProvider(mockRepository, mockMetadata, mockLifecycle, lock); + } + + public void testConstructor() { + assertNotNull(provider); + // Verify that the provider extends BlobStoreProvider + assertTrue(provider instanceof BlobStoreProvider); + } + + public void testGetBlobStoreWithClientSideEncryption() throws Exception { + // Setup: Mock the serverSideEncryptedBlobStore to return a value + // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createBlobStore()).thenReturn(mockBlobStore); + + // Initialize the server-side encrypted blob store + provider.blobStore(false); + + // Test + BlobStore result = provider.getBlobStore(false); + + // Verify + assertEquals(mockBlobStore, result); + } + + public void testGetBlobStoreWithServerSideEncryption() throws Exception { + // Setup: Mock the serverSideEncryptedBlobStore to return a value + // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createBlobStore()).thenReturn(mockBlobStore); + + // Test + expectThrows(IllegalArgumentException.class, () -> provider.getBlobStore(true)); + + } + + public void testBlobStoreWithClientSideEncryptionFirstTime() throws Exception { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createBlobStore()).thenReturn(mockBlobStore); + + // Test + BlobStore result = provider.blobStore(false); + + // Verify + assertEquals(mockBlobStore, result); + verify(mockRepository).createBlobStore(); + } + + public void testBlobStoreWithClientSideEncryptionEnabledSubsequentCalls() throws Exception { + // Setup + when(mockLifecycle.started()).thenReturn(true); + when(mockRepository.createBlobStore()).thenReturn(mockBlobStore); + + // First call + BlobStore firstResult = provider.blobStore(false); + + // Second call + BlobStore secondResult = provider.blobStore(false); + + // Verify + assertEquals(mockBlobStore, firstResult); + assertEquals(mockBlobStore, secondResult); + assertSame(firstResult, secondResult); + // Verify createServerSideEncryptedBlobStore is called only once + verify(mockRepository, times(1)).createBlobStore(); + } + + public void testInitBlobStoreWhenLifecycleNotStarted() { + // Setup + when(mockLifecycle.started()).thenReturn(false); + when(mockLifecycle.state()).thenReturn(Lifecycle.State.STOPPED); + + // Test - should throw RepositoryException + expectThrows(RepositoryException.class, () -> provider.initBlobStore(false)); + } +} diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java index 8522970391e63..78d7da2f22167 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java @@ -8,18 +8,20 @@ package org.opensearch.repositories.blobstore; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.repositories.RepositoryException; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; -import static org.mockito.Mockito.*; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; -import org.opensearch.test.OpenSearchTestCase; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class ServerSideEncryptionEnabledBlobStoreProviderTests extends OpenSearchTestCase { @@ -50,22 +52,15 @@ public void setUp() throws Exception { MockitoAnnotations.openMocks(this); lock = new Object(); when(mockMetadata.name()).thenReturn("test-repository"); - provider = new ServerSideEncryptionEnabledBlobStoreProvider( - mockRepository, - mockMetadata, - mockLifecycle, - lock - ); + provider = new ServerSideEncryptionEnabledBlobStoreProvider(mockRepository, mockMetadata, mockLifecycle, lock); } - @Test public void testConstructor() { assertNotNull(provider); // Verify that the provider extends BlobStoreProvider assertTrue(provider instanceof BlobStoreProvider); } - @Test public void testGetBlobStoreWithServerSideEncryptionEnabled() { // Setup: Mock the serverSideEncryptedBlobStore to return a value // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it @@ -82,7 +77,6 @@ public void testGetBlobStoreWithServerSideEncryptionEnabled() { assertEquals(mockServerSideEncryptedBlobStore, result); } - @Test public void testGetBlobStoreWithServerSideEncryptionDisabled() { // Setup: Mock the regular blobStore when(mockLifecycle.started()).thenReturn(true); @@ -98,7 +92,6 @@ public void testGetBlobStoreWithServerSideEncryptionDisabled() { assertEquals(mockRegularBlobStore, result); } - @Test public void testBlobStoreWithServerSideEncryptionEnabledFirstTime() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -112,7 +105,6 @@ public void testBlobStoreWithServerSideEncryptionEnabledFirstTime() { verify(mockRepository).createServerSideEncryptedBlobStore(); } - @Test public void testBlobStoreWithServerSideEncryptionEnabledSubsequentCalls() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -132,7 +124,6 @@ public void testBlobStoreWithServerSideEncryptionEnabledSubsequentCalls() { verify(mockRepository, times(1)).createServerSideEncryptedBlobStore(); } - @Test public void testBlobStoreWithServerSideEncryptionDisabled() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -145,7 +136,6 @@ public void testBlobStoreWithServerSideEncryptionDisabled() { assertEquals(mockClientSideEncryptedBlobStore, result); } - @Test public void testInitBlobStoreWithServerSideEncryptionEnabled() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -159,7 +149,6 @@ public void testInitBlobStoreWithServerSideEncryptionEnabled() { verify(mockRepository).createServerSideEncryptedBlobStore(); } - @Test public void testInitBlobStoreWithServerSideEncryptionDisabled() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -173,17 +162,15 @@ public void testInitBlobStoreWithServerSideEncryptionDisabled() { verify(mockRepository).createClientSideEncryptedBlobStore(); } - @Test(expected = RepositoryException.class) public void testInitBlobStoreWhenLifecycleNotStarted() { // Setup when(mockLifecycle.started()).thenReturn(false); when(mockLifecycle.state()).thenReturn(Lifecycle.State.STOPPED); // Test - should throw RepositoryException - provider.initBlobStore(true); + expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); } - @Test(expected = RepositoryException.class) public void testInitBlobStoreWhenRepositoryThrowsRepositoryException() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -191,10 +178,9 @@ public void testInitBlobStoreWhenRepositoryThrowsRepositoryException() { when(mockRepository.createServerSideEncryptedBlobStore()).thenThrow(repositoryException); // Test - should re-throw RepositoryException - provider.initBlobStore(true); + expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); } - @Test(expected = RepositoryException.class) public void testInitBlobStoreWhenRepositoryThrowsGenericException() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -203,16 +189,14 @@ public void testInitBlobStoreWhenRepositoryThrowsGenericException() { // Test - should wrap in RepositoryException try { - provider.initBlobStore(true); + expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); } catch (RepositoryException e) { assertEquals("test-repository", e.repository()); assertEquals("[test-repository] cannot create blob store", e.getMessage()); assertEquals(genericException, e.getCause()); - throw e; } } - @Test public void testCloseWithServerSideEncryptedBlobStore() throws Exception { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -228,7 +212,6 @@ public void testCloseWithServerSideEncryptedBlobStore() throws Exception { verify(mockServerSideEncryptedBlobStore).close(); } - @Test public void testCloseWithoutServerSideEncryptedBlobStore() throws Exception { // Test - close without initializing server-side encrypted blob store provider.close(); @@ -237,7 +220,6 @@ public void testCloseWithoutServerSideEncryptedBlobStore() throws Exception { // This test passes if no exception is thrown } - @Test public void testCloseWhenServerSideEncryptedBlobStoreThrowsException() throws Exception { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -255,7 +237,6 @@ public void testCloseWhenServerSideEncryptedBlobStoreThrowsException() throws Ex // Test passes if no exception is propagated } - @Test public void testMixedUsageServerSideAndClientSide() { // Setup when(mockLifecycle.started()).thenReturn(true); @@ -275,7 +256,6 @@ public void testMixedUsageServerSideAndClientSide() { verify(mockRepository).createClientSideEncryptedBlobStore(); } - @Test public void testGetBlobStoreReturnsNullWhenNotInitialized() { // Test - getBlobStore when server-side encrypted store is not initialized BlobStore result = provider.getBlobStore(true); From 3560a0a3cb4e97c3c6dd3fd34b1f5fb5991ae325 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Thu, 16 Oct 2025 10:48:17 +0530 Subject: [PATCH 09/10] Add comments Signed-off-by: Pranit Kumar --- .../repositories/blobstore/BlobStoreProviderTests.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java index 9bf3c0d3a4b8c..6a4da476a136c 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java @@ -22,6 +22,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +/** + * Test class for {@link BlobStoreProvider}. + */ public class BlobStoreProviderTests extends OpenSearchTestCase { @Mock private BlobStoreRepository mockRepository; From cca747420fb42521ee523174e6dd3b19d5119401 Mon Sep 17 00:00:00 2001 From: Pranit Kumar Date: Sat, 18 Oct 2025 18:37:05 +0530 Subject: [PATCH 10/10] Address comments Signed-off-by: Pranit Kumar --- .../repositories/s3/S3Repository.java | 56 ---- .../repositories/s3/S3RepositoryTests.java | 34 --- .../TransportRemoteStoreMetadataAction.java | 4 +- .../cluster/metadata/IndexMetadata.java | 36 +-- .../metadata/MetadataCreateIndexService.java | 66 ++++- .../common/settings/IndexScopedSettings.java | 1 - .../org/opensearch/index/IndexService.java | 5 +- .../org/opensearch/index/IndexSettings.java | 21 +- .../index/remote/RemoteStoreUtils.java | 5 + .../opensearch/index/shard/IndexShard.java | 5 +- .../opensearch/index/shard/StoreRecovery.java | 6 +- .../RemoteSegmentStoreDirectoryFactory.java | 4 +- .../opensearch/indices/IndicesService.java | 5 +- .../remotestore/RemoteStoreNodeAttribute.java | 35 +-- .../blobstore/BlobStoreProvider.java | 26 +- .../blobstore/BlobStoreProviderFactory.java | 52 ---- .../blobstore/BlobStoreRepository.java | 25 +- ...ideEncryptionEnabledBlobStoreProvider.java | 88 ------ .../opensearch/snapshots/RestoreService.java | 5 +- .../MetadataCreateIndexServiceTests.java | 162 ++++++----- .../opensearch/index/IndexModuleTests.java | 3 +- .../BlobStoreProviderFactoryTests.java | 210 -------------- .../blobstore/BlobStoreProviderTests.java | 19 +- ...cryptionEnabledBlobStoreProviderTests.java | 266 ------------------ .../index/shard/IndexShardTestCase.java | 3 +- 25 files changed, 221 insertions(+), 921 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java delete mode 100644 server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java delete mode 100644 server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java delete mode 100644 server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java 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 6bbad99a45d7d..8d8de283f75bb 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 @@ -507,62 +507,6 @@ protected S3BlobStore createBlobStore() { ); } - @Override - protected S3BlobStore createClientSideEncryptedBlobStore() { - serverSideEncryptionType = ServerSideEncryption.AES256.toString(); - return new S3BlobStore( - service, - s3AsyncService, - multipartUploadEnabled, - bucket, - bufferSize, - cannedACL, - storageClass, - bulkDeletesSize, - metadata, - asyncUploadUtils, - urgentExecutorBuilder, - priorityExecutorBuilder, - normalExecutorBuilder, - normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ, - genericStatsMetricPublisher, - serverSideEncryptionType, - null, - false, - null, - null - ); - } - - @Override - protected S3BlobStore createServerSideEncryptedBlobStore() { - serverSideEncryptionType = ServerSideEncryption.AWS_KMS.toString(); - return new S3BlobStore( - service, - s3AsyncService, - multipartUploadEnabled, - bucket, - bufferSize, - cannedACL, - storageClass, - bulkDeletesSize, - metadata, - asyncUploadUtils, - urgentExecutorBuilder, - priorityExecutorBuilder, - normalExecutorBuilder, - normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ, - genericStatsMetricPublisher, - serverSideEncryptionType, - serverSideEncryptionKmsKey, - serverSideEncryptionBucketKey, - serverSideEncryptionEncryptionContext, - expectedBucketOwner - ); - } - // only use for testing (S3RepositoryTests) @Override protected BlobStore getBlobStore() { 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 1c29f8178dff3..49c6a31e32816 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,7 +33,6 @@ 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; @@ -176,39 +175,6 @@ public void testValidateHttpLClientType_Invalid_Values() { } } - public void testCreateClientSideEncryptedBlobStore() { - final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", Settings.EMPTY); - try (S3Repository s3Repo = createS3Repo(metadata)) { - // Don't expect any Exception - S3BlobStore blobStore = s3Repo.createClientSideEncryptedBlobStore(); - assertNotNull(blobStore); - assertNull(blobStore.serverSideEncryptionKmsKey()); - assertEquals(blobStore.serverSideEncryptionType(), ServerSideEncryption.AES256.toString()); - assertNull(blobStore.expectedBucketOwner()); - assertFalse(blobStore.serverSideEncryptionBucketKey()); - } - } - - public void testCreateServerSideEncryptedBlobStore() { - Settings settings = Settings.builder() - .put("server_side_encryption_type", ServerSideEncryption.AWS_KMS.toString()) - .put("server_side_encryption_kms_key_id", "kms-key-id") - .put("server_side_encryption_bucket_key_enabled", true) - .put("server_side_encryption_encryption_context", "enc-ctx") - .put("expected_bucket_owner", "123456789012") - .build(); - - final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", settings); - try (S3Repository s3Repo = createS3Repo(metadata)) { - // Don't expect any Exception - S3BlobStore blobStore = s3Repo.createServerSideEncryptedBlobStore(); - assertNotNull(blobStore); - assertEquals("kms-key-id", blobStore.serverSideEncryptionKmsKey()); - assertEquals("123456789012", blobStore.expectedBucketOwner()); - assertTrue(blobStore.serverSideEncryptionBucketKey()); - } - } - 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 e0773130653fd..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 @@ -200,7 +200,7 @@ private Map> getSegmentMetadata( shardId, indexSettings.getRemoteStorePathStrategy(), null, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); Map segmentMetadataMapWithFilenames = remoteDirectory.readLatestNMetadataFiles(5); @@ -260,7 +260,7 @@ private Map> getTranslogMetadataFiles( indexSettings.getRemoteStorePathStrategy(), new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()), RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata), - indexSettings.isServerSideEncryptionEnabled() + 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 1925322e4cbbc..a5f84d7101855 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -371,7 +371,7 @@ public Iterator> settings() { ); public static final String SETTING_REMOTE_STORE_ENABLED = "index.remote_store.enabled"; - public static final String SETTING_REMOTE_STORE_SSE_ENABLED = "index.remote_store.sse.enabled"; + // public static final String SETTING_REMOTE_STORE_SSE_ENABLED = "index.remote_store.sse.enabled"; public static final String SETTING_INDEX_APPEND_ONLY_ENABLED = "index.append_only.enabled"; public static final String SETTING_REMOTE_SEGMENT_STORE_REPOSITORY = "index.remote_store.segment.repository"; @@ -415,38 +415,6 @@ public Iterator> settings() { Property.Dynamic ); - /** - * Used to specify if the index data should be persisted in the remote store. - */ - public static final Setting INDEX_REMOTE_STORE_SSE_ENABLED_SETTING = Setting.boolSetting( - SETTING_REMOTE_STORE_SSE_ENABLED, - false, - new Setting.Validator<>() { - - @Override - public void validate(final Boolean value) {} - - @Override - public void validate(final Boolean value, final Map, Object> settings) { - final Boolean isRemoteStoreEnabled = (Boolean) settings.get(INDEX_REMOTE_STORE_ENABLED_SETTING); - if (!isRemoteStoreEnabled && value) { - throw new IllegalArgumentException( - "Server Side Encryption can be enabled when " + INDEX_REMOTE_STORE_ENABLED_SETTING.getKey() + " is enabled. " - ); - } - } - - @Override - public Iterator> settings() { - final List> settings = List.of(INDEX_REMOTE_STORE_ENABLED_SETTING); - return settings.iterator(); - } - }, - Property.IndexScope, - Property.PrivateIndex, - Property.Dynamic - ); - /** * Used to specify if the index data should be persisted in the remote store. */ @@ -1025,7 +993,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 SERVER_SIDE_ENCRYPTION_ENABLED = "server_side_encryption_enabled"; + 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 b2ec6f3228019..3a25fc8a66c33 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,27 @@ 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); + } + + if (remoteNode.isPresent() + && !isRestoreFromSnapshot + && RemoteStoreSettings.isServerSideEncryptionRepoEnabled(clusterState.nodes().getMinNodeVersion()) + && RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes())) { + 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 +754,9 @@ private ClusterState applyCreateIndexRequestWithV1Templates( clusterService.getClusterSettings() ); int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null); - IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards); + IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards, currentState); + + // buildServerSideEncryptionSetting(tmpImd, clusterService.getClusterSettings(), false); return applyCreateIndexWithTemporaryService( currentState, @@ -755,6 +781,23 @@ private ClusterState applyCreateIndexRequestWithV1Templates( ); } + // private void buildServerSideEncryptionSetting(IndexMetadata tmpImd, ClusterSettings clusterSettings, boolean isRestore) { + // + // if (isRestore) + // if (remoteStoreCustomMetadataResolver == null) { + // return; + // } + // + // Map remoteCustomMetadata = tmpImd.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + // + // String sseEnabledIndex = existingCustomData == null ? null : existingCustomData.get(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED); + // if (assertNullOldType || sseEnabledIndex != null) { + // + // } + // remoteCustomData.put(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, Boolean.toString(true)); + // + // } + private ClusterState applyCreateIndexRequestWithV2Template( final ClusterState currentState, final CreateIndexClusterStateUpdateRequest request, @@ -795,7 +838,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 +922,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, @@ -1056,7 +1099,7 @@ static Settings aggregateIndexSettings( indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings, combinedTemplateSettings, clusterSettings); - updateRemoteStoreSettings(indexSettingsBuilder, currentState, clusterSettings, settings, request.index(), false); + updateRemoteStoreSettings(indexSettingsBuilder, currentState, clusterSettings, settings, request.index()); if (sourceMetadata != null) { assert request.resizeType() != null; @@ -1162,8 +1205,7 @@ public static void updateRemoteStoreSettings( ClusterState clusterState, ClusterSettings clusterSettings, Settings nodeSettings, - String indexName, - boolean isRestoreFromSnapshot + String indexName ) { if ((isRemoteDataAttributePresent(nodeSettings) && clusterSettings.get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(RemoteStoreNodeService.CompatibilityMode.STRICT)) @@ -1179,12 +1221,6 @@ public static void updateRemoteStoreSettings( if (remoteNode.isPresent()) { segmentRepo = RemoteStoreNodeAttribute.getSegmentRepoName(remoteNode.get().getAttributes()); - if (!isRestoreFromSnapshot - && RemoteStoreSettings.isServerSideEncryptionRepoEnabled(clusterState.nodes().getMinNodeVersion()) - && RemoteStoreNodeAttribute.isRemoteStoreServerSideEncryptionEnabled(remoteNode.get().getAttributes(), segmentRepo)) { - settingsBuilder.put(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, true); - } - translogRepo = RemoteStoreNodeAttribute.getTranslogRepoName(remoteNode.get().getAttributes()); if (segmentRepo != null) { settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true).put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepo); diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index f595edc3bb93e..7bbcdec25ce12 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -242,7 +242,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings { // Settings for remote store enablement IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, - IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING, diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index d73f8f5e5d9e8..9c1bcf77f7db2 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; @@ -697,12 +698,12 @@ public synchronized IndexShard createShard( } remoteDirectory = ((RemoteSegmentStoreDirectoryFactory) remoteDirectoryFactory).newDirectory( - RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getSettings()), + RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getNodeSettings()), this.indexSettings.getUUID(), shardId, this.indexSettings.getRemoteStorePathStrategy(), this.indexSettings.getRemoteStoreSegmentPathPrefix(), - this.indexSettings.isServerSideEncryptionEnabled() + 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. diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 7f27aea2a1447..bf3fef3d83caf 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -950,8 +950,6 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { */ private final boolean isCompositeIndex; - private boolean isServerSideEncryptionEnabled; - /** * Denotes whether search via star tree index is enabled for this index */ @@ -1037,7 +1035,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null); replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); - isServerSideEncryptionEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false); + isWarmIndex = settings.getAsBoolean(IndexModule.IS_WARM_INDEX_SETTING.getKey(), false); remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); @@ -1245,12 +1243,6 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti ); scopedSettings.addSettingsUpdateConsumer(ALLOW_DERIVED_FIELDS, this::setAllowDerivedField); scopedSettings.addSettingsUpdateConsumer(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, this::setRemoteStoreEnabled); - - scopedSettings.addSettingsUpdateConsumer( - IndexMetadata.INDEX_REMOTE_STORE_SSE_ENABLED_SETTING, - this::setServerSideEncryptionEnabled - ); - scopedSettings.addSettingsUpdateConsumer( IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, this::setRemoteStoreRepository @@ -1439,13 +1431,6 @@ public boolean isRemoteStoreEnabled() { return isRemoteStoreEnabled; } - /** - * Returns if remote store SSE is enabled for this index. - */ - public boolean isServerSideEncryptionEnabled() { - return isServerSideEncryptionEnabled; - } - public boolean isAssignedOnRemoteNode() { return assignedOnRemoteNode; } @@ -2156,10 +2141,6 @@ public void setRemoteStoreEnabled(boolean isRemoteStoreEnabled) { this.isRemoteStoreEnabled = isRemoteStoreEnabled; } - public void setServerSideEncryptionEnabled(boolean sseEnabled) { - this.isServerSideEncryptionEnabled = sseEnabled; - } - public void setRemoteStoreRepository(String remoteStoreRepository) { this.remoteStoreRepository = remoteStoreRepository; } 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 767f8a115b19e..8c33c9e0763ff 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -162,6 +162,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.ReplicationTracker; @@ -5290,7 +5291,7 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings, indexSettings().isTranslogMetadataEnabled(), - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } @@ -5314,7 +5315,7 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), indexSettings().isTranslogMetadataEnabled(), 0, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 50e4ce70cc0d5..80f91d421db97 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -425,7 +425,7 @@ void recoverFromSnapshotAndRemoteStore( shardId, shallowCopyShardMetadata.getRemoteStorePathStrategy(), null, - indexShard.indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexShard.indexSettings.getIndexMetadata()) ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, @@ -509,7 +509,7 @@ void recoverShallowSnapshotV2( shardId, remoteStorePathStrategy, null, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexShard.indexSettings.getIndexMetadata()) ); RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( recoverySource.pinnedTimestamp() @@ -530,7 +530,7 @@ void recoverShallowSnapshotV2( remoteStorePathStrategy, RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), recoverySource.pinnedTimestamp(), - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexShard.indexSettings.getIndexMetadata()) ); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 83c7ea285d6c5..57455d3bbdd7c 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -14,6 +14,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; @@ -60,13 +61,14 @@ public RemoteSegmentStoreDirectoryFactory( public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException { String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); + return newDirectory( repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathStrategy(), null, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 6bb40fcd908b8..417e408f510d1 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -132,6 +132,7 @@ import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.seqno.RetentionLeaseStats; import org.opensearch.index.seqno.RetentionLeaseSyncer; @@ -729,7 +730,7 @@ private static BiFunction getTrans indexSettings.getRemoteStoreTranslogRepository(), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), remoteStoreSettings, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } else if (RemoteStoreNodeAttribute.isTranslogRepoConfigured(settings) && shardRouting.primary()) { return new RemoteBlobStoreInternalTranslogFactory( @@ -738,7 +739,7 @@ private static BiFunction getTrans RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo(indexSettings.getNodeSettings()), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()), remoteStoreSettings, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } return new InternalTranslogFactory(); diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 0c010db22d026..23f45d48aff01 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -44,7 +44,6 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.segment.repository"; public static final String REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.translog.repository"; public static final String REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.routing_table.repository"; - public static final String REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY = "server_side_encryption_enabled"; public static final List REMOTE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEYS = REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX.stream() .map(prefix -> prefix + ".state.repository") @@ -69,8 +68,9 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX = "remote_store.repository.%s.settings."; public static final String REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s.type"; + public static final String REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT = - REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX + ".server_side_encryption_enabled"; + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX + "server_side_encryption_kms_key_id"; public static final String REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT = "%s.repository.%s." + CryptoMetadata.CRYPTO_METADATA_KEY; public static final String REPOSITORY_CRYPTO_SETTINGS_PREFIX = REPOSITORY_CRYPTO_ATTRIBUTE_KEY_FORMAT @@ -253,14 +253,14 @@ public static boolean isRemoteStoreAttributePresent(Settings settings) { return false; } - public static boolean isServerSideEncryptionEnabled(Settings settings) { - for (String metadataKey : settings.keySet()) { - if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY) && settings.getAsBoolean(metadataKey, false)) { - return true; - } - } - return false; - } + // public static boolean isServerSideEncryptionEnabled(Settings settings) { + // for (String metadataKey : settings.keySet()) { + // if (metadataKey.equals(REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY) && settings.getAsBoolean(metadataKey, false)) { + // return true; + // } + // } + // return false; + // } public static boolean isRemoteDataAttributePresent(Settings settings) { return isSegmentRepoConfigured(settings) || isTranslogRepoConfigured(settings); @@ -360,13 +360,6 @@ private static boolean remoteDataAttributesPresent(Map nodeAttrs return true; } - public static boolean isRemoteStoreServerSideEncryptionEnabled(Map nodeAttributes, String repoName) { - - String attributeKey = String.format(Locale.getDefault(), REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, repoName); - String attributeValue = nodeAttributes.get(attributeKey); - return "true".equalsIgnoreCase(attributeValue); - } - public static String getClusterStateRepoName(Map repos) { return getValueFromAnyKey(repos, REMOTE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEYS); } @@ -383,6 +376,14 @@ public static String getTranslogRepoName(Map repos) { return getValueFromAnyKey(repos, REMOTE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEYS); } + public static boolean isRemoteStoreServerSideEncryptionEnabled(Map repos) { + String segmentRepoName = getSegmentRepoName(repos); + return getValueFromAnyKey( + repos, + List.of(String.format(Locale.getDefault(), REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, segmentRepoName)) + ) != null; + } + private static String getValueFromAnyKey(Map repos, List keys) { for (String key : keys) { if (repos.get(key) != null) { diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java index 714eee8745f4f..463d573a63d22 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProvider.java @@ -28,7 +28,8 @@ public class BlobStoreProvider { protected final RepositoryMetadata metadata; protected final Object lock; protected final BlobStoreRepository repository; - protected final SetOnce blobStore = new SetOnce<>(); + private final SetOnce blobStore = new SetOnce<>(); + private final SetOnce serverSideEncryptedBlobStore = new SetOnce<>(); public BlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { this.lifecycle = lifecycle; @@ -38,7 +39,12 @@ public BlobStoreProvider(BlobStoreRepository repository, RepositoryMetadata meta } protected BlobStore blobStore(boolean serverSideEncryptionEnabled) { - return createBlobStore(blobStore, serverSideEncryptionEnabled); + if (serverSideEncryptionEnabled) { + logger.info("[BlobStoreProvider]: Using serverSideEncryptedBlobStore"); + return createBlobStore(serverSideEncryptedBlobStore, true); + } + logger.info("[BlobStoreProvider]: Using blobStore"); + return createBlobStore(blobStore, false); } public BlobStore blobStore() { @@ -54,7 +60,7 @@ protected BlobStore createBlobStore(SetOnce blobStore, boolean server synchronized (lock) { store = blobStore.get(); if (store == null) { - store = initBlobStore(serverSideEncryption); + store = initBlobStore(); if (!serverSideEncryption && metadata.cryptoMetadata() != null) { store = new EncryptedBlobStore(store, metadata.cryptoMetadata()); } @@ -66,17 +72,10 @@ protected BlobStore createBlobStore(SetOnce blobStore, boolean server } public BlobStore getBlobStore(boolean serverSideEncryptionEnabled) { - if (serverSideEncryptionEnabled) { - throw new IllegalArgumentException("Provider Instance Type is not correct"); - } - return blobStore.get(); + return blobStore(serverSideEncryptionEnabled); } - // public BlobStore getBlobStore() { - // return blobStore.get(); - // } - - protected BlobStore initBlobStore(boolean serverSideEncryption) { + protected BlobStore initBlobStore() { if (lifecycle.started() == false) { throw new RepositoryException(metadata.name(), "repository is not in started state" + lifecycle.state()); } @@ -94,6 +93,9 @@ public void close() { if (blobStore.get() != null) { blobStore.get().close(); } + if (serverSideEncryptedBlobStore.get() != null) { + serverSideEncryptedBlobStore.get().close(); + } } catch (Exception t) { logger.warn("cannot close blob store", t); } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java deleted file mode 100644 index 41a13e596c24e..0000000000000 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactory.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.blobstore; - -import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.SetOnce; -import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; - -/** - * Factory class for BlobStoreProvider. - */ -public class BlobStoreProviderFactory { - private final Lifecycle lifecycle; - private final RepositoryMetadata metadata; - private final Object lock; - private final BlobStoreRepository repository; - private final SetOnce serverSideEncryptionEnabledBlobStoreProvider = new SetOnce<>(); - private final SetOnce blobStoreProvider = new SetOnce<>(); - - public BlobStoreProviderFactory(BlobStoreRepository repository, RepositoryMetadata metadata, Lifecycle lifecycle, Object lock) { - this.lifecycle = lifecycle; - this.metadata = metadata; - this.lock = lock; - this.repository = repository; - } - - public BlobStoreProvider getBlobStoreProvider() { - synchronized (lock) { - if (RemoteStoreNodeAttribute.isServerSideEncryptionEnabled(metadata.settings())) { - if (serverSideEncryptionEnabledBlobStoreProvider.get() == null) { - ServerSideEncryptionEnabledBlobStoreProvider serverSideEncryptionEnabledBlobStoreProvider = - new ServerSideEncryptionEnabledBlobStoreProvider(repository, metadata, lifecycle, lock); - this.serverSideEncryptionEnabledBlobStoreProvider.set(serverSideEncryptionEnabledBlobStoreProvider); - } - return serverSideEncryptionEnabledBlobStoreProvider.get(); - } else { - if (blobStoreProvider.get() == null) { - BlobStoreProvider blobStoreProvider = new BlobStoreProvider(repository, metadata, lifecycle, lock); - this.blobStoreProvider.set(blobStoreProvider); - } - return blobStoreProvider.get(); - } - } - } -} diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 888245623c697..605cf52ed0879 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -566,7 +566,6 @@ protected static long calculateMaxWithinIntLimit(long defaultThresholdOfHeap, lo private final SetOnce snapshotShardPathBlobContainer = new SetOnce<>(); - private final SetOnce blobStoreProvideFactory = new SetOnce<>(); private final SetOnce blobStoreProvider = new SetOnce<>(); protected final ClusterService clusterService; @@ -1064,25 +1063,15 @@ public BlobStore blobStore() { * @return BlobStore `Blobstore` for the repository */ public BlobStore blobStore(boolean serverSideEncryptionEnabled) { - BlobStoreProviderFactory providerFactory = this.blobStoreProvideFactory.get(); - logger.info("providerFactory = " + providerFactory); - if (providerFactory == null) { - synchronized (lock) { - providerFactory = new BlobStoreProviderFactory(this, metadata, lifecycle, lock); - blobStoreProvideFactory.set(providerFactory); - } - } - logger.info("2.providerFactory = " + providerFactory); - BlobStoreProvider provider = this.blobStoreProvider.get(); - logger.info("1.provider = " + provider); + logger.info("1. provider = " + provider); if (provider == null) { synchronized (lock) { - provider = providerFactory.getBlobStoreProvider(); + provider = new BlobStoreProvider(this, metadata, lifecycle, lock); this.blobStoreProvider.set(provider); } } - logger.info("2.provider = " + provider); + logger.info("provider = " + provider); return provider.blobStore(serverSideEncryptionEnabled); } @@ -1091,14 +1080,6 @@ public BlobStore blobStore(boolean serverSideEncryptionEnabled) { */ protected abstract BlobStore createBlobStore() throws Exception; - protected BlobStore createServerSideEncryptedBlobStore() { - throw new UnsupportedOperationException(); - } - - protected BlobStore createClientSideEncryptedBlobStore() { - throw new UnsupportedOperationException(); - } - /** * Returns base path of the repository */ diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java b/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java deleted file mode 100644 index 7181e6f07118b..0000000000000 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProvider.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.blobstore; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.SetOnce; -import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.repositories.RepositoryException; - -/** - * BlobStoreProvider for RemoteStoreProvider - */ -public class ServerSideEncryptionEnabledBlobStoreProvider extends BlobStoreProvider { - private static final Logger logger = LogManager.getLogger(ServerSideEncryptionEnabledBlobStoreProvider.class); - private final SetOnce serverSideEncryptedBlobStore = new SetOnce<>(); - - public ServerSideEncryptionEnabledBlobStoreProvider( - BlobStoreRepository repository, - RepositoryMetadata metadata, - Lifecycle lifecycle, - Object lock - ) { - super(repository, metadata, lifecycle, lock); - } - - public BlobStore getBlobStore(boolean serverSideEncryptionEnabled) { - if (serverSideEncryptionEnabled) { - return serverSideEncryptedBlobStore.get(); - } - return blobStore.get(); - } - - /** - * - */ - public BlobStore blobStore(boolean serverSideEncryptionEnabled) { - BlobStore store = null; - if (serverSideEncryptionEnabled) { - store = serverSideEncryptedBlobStore.get(); - if (store == null) { - store = super.createBlobStore(serverSideEncryptedBlobStore, true); - } - } else { - store = super.blobStore(false); - } - return store; - } - - /** - * - */ - protected BlobStore initBlobStore(boolean serverSideEncryptionEnabled) { - if (lifecycle.started() == false) { - throw new RepositoryException(metadata.name(), "repository is not in started state" + lifecycle.state()); - } - try { - if (serverSideEncryptionEnabled) { - return repository.createServerSideEncryptedBlobStore(); - } else { - return repository.createClientSideEncryptedBlobStore(); - } - } catch (RepositoryException e) { - throw e; - } catch (Exception e) { - throw new RepositoryException(metadata.name(), "cannot create blob store", e); - } - } - - public void close() { - super.close(); - try { - if (serverSideEncryptedBlobStore.get() != null) { - serverSideEncryptedBlobStore.get().close(); - } - } catch (Exception t) { - logger.warn("cannot close blob store", t); - } - } -} diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index e590bdff6918d..ef4d0df2e9636 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -469,7 +469,7 @@ public ClusterState execute(ClusterState currentState) { .put(snapshotIndexMetadata.getSettings()) .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) ); - createIndexService.addRemoteStoreCustomMetadata(indexMdBuilder, false); + createIndexService.addRemoteStoreCustomMetadata(indexMdBuilder, false, currentState); shardLimitValidator.validateShardLimit( renamedIndexName, snapshotIndexMetadata.getSettings(), @@ -722,8 +722,7 @@ private Settings getOverrideSettingsInternal() { clusterService.state(), clusterSettings, clusterService.getSettings(), - String.join(",", request.indices()), - true + String.join(",", request.indices()) ); return settingsBuilder.build(); } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index fbd172be9a15b..a75ada3caab9c 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -175,7 +175,6 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; @@ -1823,7 +1822,12 @@ private IndexMetadata testRemoteCustomData(boolean remoteStoreEnabled, PathType .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build(); - IndexMetadata indexMetadata = metadataCreateIndexService.buildAndValidateTemporaryIndexMetadata(indexSettings, request, 0); + IndexMetadata indexMetadata = metadataCreateIndexService.buildAndValidateTemporaryIndexMetadata( + indexSettings, + request, + 0, + clusterService.state() + ); threadPool.shutdown(); return indexMetadata; } @@ -1863,7 +1867,8 @@ public void testNumberOfRoutingShardsShowsInIndexSettings() { IndexMetadata indexMetadata = checkerService.buildAndValidateTemporaryIndexMetadata( indexSettings, request, - routingNumberOfShards + routingNumberOfShards, + clusterService.state() ); assertEquals(INDEX_NUMBER_OF_ROUTING_SHARDS_SETTING.get(indexMetadata.getSettings()).intValue(), routingNumberOfShards); })); @@ -2617,67 +2622,95 @@ public void testIndexTotalPrimaryShardsPerNodeSettingValidationWithoutRemoteStor ); } - public void testUpdateRemoteStoreSettings() { - Map attributes = getNodeAttributes(); - DiscoveryNode remoteNode = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - attributes, - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) - .nodes(DiscoveryNodes.builder().add(remoteNode).build()) - .build(); - - Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); - final Settings.Builder requestSettings = Settings.builder() - .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); - - Settings clusterSettingsSetting = Settings.builder() - .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) - .build(); - clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - - new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); - - MetadataCreateIndexService.updateRemoteStoreSettings(requestSettings, clusterState, clusterSettings, settings, "test-index", false); - - assertTrue(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); - } - - public void testUpdateRemoteStoreSettings_For_Snapshot_restore() { - Map attributes = getNodeAttributes(); - DiscoveryNode remoteNode = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - attributes, - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) - .nodes(DiscoveryNodes.builder().add(remoteNode).build()) - .build(); - - Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); - final Settings.Builder requestSettings = Settings.builder() - .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); - - Settings clusterSettingsSetting = Settings.builder() - .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) - .build(); - clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - - new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); - - MetadataCreateIndexService.updateRemoteStoreSettings(requestSettings, clusterState, clusterSettings, settings, "test-index", true); - - assertFalse(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); - } + // public void testAddRemoteStoreCustomMetadata() { + // + // MetadataCreateIndexService checkerService = new MetadataCreateIndexService( + // Settings.EMPTY, + // clusterService, + // indicesServices, + // null, + // null, + // createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), + // null, + // null, + // null, + // null, + // new SystemIndices(Collections.emptyMap()), + // false, + // new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), + // DefaultRemoteStoreSettings.INSTANCE, + // repositoriesServiceSupplier + // ); + // + // + // Map attributes = getNodeAttributes(); + // DiscoveryNode remoteNode = new DiscoveryNode( + // UUIDs.base64UUID(), + // buildNewFakeTransportAddress(), + // attributes, + // DiscoveryNodeRole.BUILT_IN_ROLES, + // Version.CURRENT + // ); + // ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + // .nodes(DiscoveryNodes.builder().add(remoteNode).build()) + // .build(); + // + // Settings indexSettings = Settings.builder() + // .put(SETTING_VERSION_CREATED, Version.CURRENT) + // .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + // .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + // .build(); + // + // IndexMetadata.Builder imdBuilder = IndexMetadata.builder("test").settings(indexSettings); + // + // Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); + // final Settings.Builder requestSettings = Settings.builder() + // .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) + // .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); + // + // Settings clusterSettingsSetting = Settings.builder() + // .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) + // .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + // .build(); + // clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + // + // new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); + // + // MetadataCreateIndexService.add(imdBuilder, ); + // + // assertTrue(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); + // } + // + // public void testUpdateRemoteStoreSettings_For_Snapshot_restore() { + // Map attributes = getNodeAttributes(); + // DiscoveryNode remoteNode = new DiscoveryNode( + // UUIDs.base64UUID(), + // buildNewFakeTransportAddress(), + // attributes, + // DiscoveryNodeRole.BUILT_IN_ROLES, + // Version.CURRENT + // ); + // ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + // .nodes(DiscoveryNodes.builder().add(remoteNode).build()) + // .build(); + // + // Settings settings = Settings.builder().put("node.attr.remote_store.segment.repository", "my-segment-repo-1").build(); + // final Settings.Builder requestSettings = Settings.builder() + // .put(INDEX_TOTAL_PRIMARY_SHARDS_PER_NODE_SETTING.getKey(), -1) + // .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT.toString()); + // + // Settings clusterSettingsSetting = Settings.builder() + // .put(RemoteStoreSettings.CLUSTER_SERVER_SIDE_ENCRYPTION_REPO_ENABLED.getKey(), true) + // .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + // .build(); + // clusterSettings = new ClusterSettings(clusterSettingsSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + // + // new RemoteStoreSettings(clusterSettingsSetting, clusterSettings); + // + // MetadataCreateIndexService.updateRemoteStoreSettings(requestSettings, clusterState, clusterSettings, settings, "test-index", true); + // + // assertFalse(requestSettings.build().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_SSE_ENABLED, false)); + // } private static Map getNodeAttributes() { String segmentRepositoryName = "my-segment-repo-1"; @@ -2784,7 +2817,6 @@ private DiscoveryNode getRemoteNode() { attributes.put(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-cluster-rep-1"); attributes.put(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-segment-repo-1"); attributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); - attributes.put(REPOSITORY_SERVER_SIDE_ENCRYPTION_ATTRIBUTE_KEY_FORMAT, "my-translog-repo-1"); return new DiscoveryNode( UUIDs.base64UUID(), buildNewFakeTransportAddress(), diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 73fe63b916dd7..8b78796b4679b 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -87,6 +87,7 @@ import org.opensearch.index.fielddata.IndexFieldDataCache; import org.opensearch.index.mapper.ParsedDocument; import org.opensearch.index.mapper.Uid; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexingOperationListener; @@ -240,7 +241,7 @@ private IndexService newIndexService(IndexModule module) throws IOException { indexSettings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 10), DefaultRemoteStoreSettings.INSTANCE, - indexSettings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(indexSettings.getIndexMetadata()) ); } return new InternalTranslogFactory(); diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java deleted file mode 100644 index e720d06b92b06..0000000000000 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderFactoryTests.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.blobstore; - -import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.common.settings.Settings; -import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; -import org.opensearch.test.OpenSearchTestCase; -import org.junit.Before; - -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class BlobStoreProviderFactoryTests extends OpenSearchTestCase { - - @Mock - private BlobStoreRepository mockRepository; - - @Mock - private RepositoryMetadata mockMetadata; - - @Mock - private Lifecycle mockLifecycle; - - private Object lock; - private BlobStoreProviderFactory factory; - - @Before - public void setUp() throws Exception { - super.setUp(); - MockitoAnnotations.initMocks(this); - lock = new Object(); - } - - public void testConstructor() { - // Test successful construction - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - assertNotNull(factory); - } - - public void testGetBlobStoreProviderWithServerSideEncryptionEnabled() { - // Setup - Settings settings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) - .build(); - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute - BlobStoreProvider provider = factory.getBlobStoreProvider(); - - // Verify - assertNotNull(provider); - assertTrue(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); - verify(mockMetadata, atLeastOnce()).settings(); - } - - public void testGetBlobStoreProviderWithServerSideEncryptionDisabled() { - // Setup - Settings settings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) - .build(); - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute - BlobStoreProvider provider = factory.getBlobStoreProvider(); - - // Verify - assertNotNull(provider); - assertFalse(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); - assertTrue(provider instanceof BlobStoreProvider); - verify(mockMetadata, atLeastOnce()).settings(); - } - - public void testGetBlobStoreProviderWithDefaultSettings() { - // Setup - empty settings (default behavior) - Settings settings = Settings.EMPTY; - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute - BlobStoreProvider provider = factory.getBlobStoreProvider(); - - // Verify - assertNotNull(provider); - assertFalse(provider instanceof ServerSideEncryptionEnabledBlobStoreProvider); - assertTrue(provider instanceof BlobStoreProvider); - } - - public void testGetBlobStoreProviderSingletonBehaviorWithEncryption() { - // Setup - Settings settings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) - .build(); - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute multiple calls - BlobStoreProvider provider1 = factory.getBlobStoreProvider(); - BlobStoreProvider provider2 = factory.getBlobStoreProvider(); - - // Verify same instance is returned (singleton behavior) - assertSame(provider1, provider2); - assertTrue(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); - } - - public void testGetBlobStoreProviderSingletonBehaviorWithoutEncryption() { - // Setup - Settings settings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) - .build(); - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute multiple calls - BlobStoreProvider provider1 = factory.getBlobStoreProvider(); - BlobStoreProvider provider2 = factory.getBlobStoreProvider(); - - // Verify same instance is returned (singleton behavior) - assertSame(provider1, provider2); - assertFalse(provider1 instanceof ServerSideEncryptionEnabledBlobStoreProvider); - } - - public void testGetBlobStoreProviderThreadSafety() throws InterruptedException { - // Setup - Settings settings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) - .build(); - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Test concurrent access - final BlobStoreProvider[] providers = new BlobStoreProvider[2]; - Thread thread1 = new Thread(() -> providers[0] = factory.getBlobStoreProvider()); - Thread thread2 = new Thread(() -> providers[1] = factory.getBlobStoreProvider()); - - thread1.start(); - thread2.start(); - - thread1.join(); - thread2.join(); - - // Verify both threads get the same instance - assertNotNull(providers[0]); - assertNotNull(providers[1]); - assertSame(providers[0], providers[1]); - } - - public void testGetBlobStoreProviderParameterPassing() { - // Setup - Settings settings = Settings.EMPTY; - when(mockMetadata.settings()).thenReturn(settings); - - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - // Execute - BlobStoreProvider provider = factory.getBlobStoreProvider(); - - // Verify that the provider was created with correct parameters - // This test ensures that the factory passes the correct constructor parameters - assertNotNull(provider); - - // Additional verification could be done if BlobStoreProvider had getter methods - // or if we could verify the constructor calls through mocking - } - - public void testFactoryStateConsistency() { - // Setup - Settings encryptedSettings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, true) - .build(); - Settings nonEncryptedSettings = Settings.builder() - .put(RemoteStoreNodeAttribute.REPOSITORY_METADATA_SERVER_SIDE_ENCRYPTION_ENABLED_KEY, false) - .build(); - - // Test that factory maintains consistent state based on initial metadata - when(mockMetadata.settings()).thenReturn(encryptedSettings); - factory = new BlobStoreProviderFactory(mockRepository, mockMetadata, mockLifecycle, lock); - - BlobStoreProvider encryptedProvider = factory.getBlobStoreProvider(); - assertTrue(encryptedProvider instanceof ServerSideEncryptionEnabledBlobStoreProvider); - - // Even if we change the mock to return different settings, - // the factory should maintain its initial behavior - when(mockMetadata.settings()).thenReturn(nonEncryptedSettings); - BlobStoreProvider sameProvider = factory.getBlobStoreProvider(); - - assertNotSame(encryptedProvider, sameProvider); - assertTrue(sameProvider != null); - } - -} diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java index 6a4da476a136c..1bd9bf16ee357 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreProviderTests.java @@ -39,7 +39,7 @@ public class BlobStoreProviderTests extends OpenSearchTestCase { private BlobStore mockBlobStore; @Mock - private BlobStore mockRegularBlobStore; + private BlobStore mockServerSideEncryptionBlobStore; private Object lock; private BlobStoreProvider provider; @@ -53,13 +53,7 @@ public void setUp() throws Exception { provider = new BlobStoreProvider(mockRepository, mockMetadata, mockLifecycle, lock); } - public void testConstructor() { - assertNotNull(provider); - // Verify that the provider extends BlobStoreProvider - assertTrue(provider instanceof BlobStoreProvider); - } - - public void testGetBlobStoreWithClientSideEncryption() throws Exception { + public void testGetBlobStore() throws Exception { // Setup: Mock the serverSideEncryptedBlobStore to return a value // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it when(mockLifecycle.started()).thenReturn(true); @@ -79,11 +73,12 @@ public void testGetBlobStoreWithServerSideEncryption() throws Exception { // Setup: Mock the serverSideEncryptedBlobStore to return a value // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createBlobStore()).thenReturn(mockBlobStore); + when(mockRepository.createBlobStore()).thenReturn(mockServerSideEncryptionBlobStore); - // Test - expectThrows(IllegalArgumentException.class, () -> provider.getBlobStore(true)); + BlobStore result = provider.getBlobStore(true); + // Verify + assertEquals(mockServerSideEncryptionBlobStore, result); } public void testBlobStoreWithClientSideEncryptionFirstTime() throws Exception { @@ -124,6 +119,6 @@ public void testInitBlobStoreWhenLifecycleNotStarted() { when(mockLifecycle.state()).thenReturn(Lifecycle.State.STOPPED); // Test - should throw RepositoryException - expectThrows(RepositoryException.class, () -> provider.initBlobStore(false)); + expectThrows(RepositoryException.class, () -> provider.initBlobStore()); } } diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java deleted file mode 100644 index 78d7da2f22167..0000000000000 --- a/server/src/test/java/org/opensearch/repositories/blobstore/ServerSideEncryptionEnabledBlobStoreProviderTests.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.blobstore; - -import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.lifecycle.Lifecycle; -import org.opensearch.repositories.RepositoryException; -import org.opensearch.test.OpenSearchTestCase; -import org.junit.Before; - -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class ServerSideEncryptionEnabledBlobStoreProviderTests extends OpenSearchTestCase { - - @Mock - private BlobStoreRepository mockRepository; - - @Mock - private RepositoryMetadata mockMetadata; - - @Mock - private Lifecycle mockLifecycle; - - @Mock - private BlobStore mockServerSideEncryptedBlobStore; - - @Mock - private BlobStore mockClientSideEncryptedBlobStore; - - @Mock - private BlobStore mockRegularBlobStore; - - private Object lock; - private ServerSideEncryptionEnabledBlobStoreProvider provider; - - @Before - public void setUp() throws Exception { - super.setUp(); - MockitoAnnotations.openMocks(this); - lock = new Object(); - when(mockMetadata.name()).thenReturn("test-repository"); - provider = new ServerSideEncryptionEnabledBlobStoreProvider(mockRepository, mockMetadata, mockLifecycle, lock); - } - - public void testConstructor() { - assertNotNull(provider); - // Verify that the provider extends BlobStoreProvider - assertTrue(provider instanceof BlobStoreProvider); - } - - public void testGetBlobStoreWithServerSideEncryptionEnabled() { - // Setup: Mock the serverSideEncryptedBlobStore to return a value - // Note: Since SetOnce is used internally, we need to first call blobStore() to initialize it - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - - // Initialize the server-side encrypted blob store - provider.blobStore(true); - - // Test - BlobStore result = provider.getBlobStore(true); - - // Verify - assertEquals(mockServerSideEncryptedBlobStore, result); - } - - public void testGetBlobStoreWithServerSideEncryptionDisabled() { - // Setup: Mock the regular blobStore - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockRegularBlobStore); - - // Initialize the regular blob store - provider.blobStore(false); - - // Test - BlobStore result = provider.getBlobStore(false); - - // Verify - assertEquals(mockRegularBlobStore, result); - } - - public void testBlobStoreWithServerSideEncryptionEnabledFirstTime() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - - // Test - BlobStore result = provider.blobStore(true); - - // Verify - assertEquals(mockServerSideEncryptedBlobStore, result); - verify(mockRepository).createServerSideEncryptedBlobStore(); - } - - public void testBlobStoreWithServerSideEncryptionEnabledSubsequentCalls() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - - // First call - BlobStore firstResult = provider.blobStore(true); - - // Second call - BlobStore secondResult = provider.blobStore(true); - - // Verify - assertEquals(mockServerSideEncryptedBlobStore, firstResult); - assertEquals(mockServerSideEncryptedBlobStore, secondResult); - assertSame(firstResult, secondResult); - // Verify createServerSideEncryptedBlobStore is called only once - verify(mockRepository, times(1)).createServerSideEncryptedBlobStore(); - } - - public void testBlobStoreWithServerSideEncryptionDisabled() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); - - // Test - BlobStore result = provider.blobStore(false); - - // Verify - assertEquals(mockClientSideEncryptedBlobStore, result); - } - - public void testInitBlobStoreWithServerSideEncryptionEnabled() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - - // Test - BlobStore result = provider.initBlobStore(true); - - // Verify - assertEquals(mockServerSideEncryptedBlobStore, result); - verify(mockRepository).createServerSideEncryptedBlobStore(); - } - - public void testInitBlobStoreWithServerSideEncryptionDisabled() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); - - // Test - BlobStore result = provider.initBlobStore(false); - - // Verify - assertEquals(mockClientSideEncryptedBlobStore, result); - verify(mockRepository).createClientSideEncryptedBlobStore(); - } - - public void testInitBlobStoreWhenLifecycleNotStarted() { - // Setup - when(mockLifecycle.started()).thenReturn(false); - when(mockLifecycle.state()).thenReturn(Lifecycle.State.STOPPED); - - // Test - should throw RepositoryException - expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); - } - - public void testInitBlobStoreWhenRepositoryThrowsRepositoryException() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - RepositoryException repositoryException = new RepositoryException("test-repo", "test error"); - when(mockRepository.createServerSideEncryptedBlobStore()).thenThrow(repositoryException); - - // Test - should re-throw RepositoryException - expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); - } - - public void testInitBlobStoreWhenRepositoryThrowsGenericException() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - RuntimeException genericException = new RuntimeException("generic error"); - when(mockRepository.createServerSideEncryptedBlobStore()).thenThrow(genericException); - - // Test - should wrap in RepositoryException - try { - expectThrows(RepositoryException.class, () -> provider.initBlobStore(true)); - } catch (RepositoryException e) { - assertEquals("test-repository", e.repository()); - assertEquals("[test-repository] cannot create blob store", e.getMessage()); - assertEquals(genericException, e.getCause()); - } - } - - public void testCloseWithServerSideEncryptedBlobStore() throws Exception { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - - // Initialize the server-side encrypted blob store - provider.blobStore(true); - - // Test - provider.close(); - - // Verify - verify(mockServerSideEncryptedBlobStore).close(); - } - - public void testCloseWithoutServerSideEncryptedBlobStore() throws Exception { - // Test - close without initializing server-side encrypted blob store - provider.close(); - - // Verify - no exception should be thrown and no close() called on null store - // This test passes if no exception is thrown - } - - public void testCloseWhenServerSideEncryptedBlobStoreThrowsException() throws Exception { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - doThrow(new RuntimeException("close error")).when(mockServerSideEncryptedBlobStore).close(); - - // Initialize the server-side encrypted blob store - provider.blobStore(true); - - // Test - should not throw exception even if blob store close() fails - provider.close(); - - // Verify - verify(mockServerSideEncryptedBlobStore).close(); - // Test passes if no exception is propagated - } - - public void testMixedUsageServerSideAndClientSide() { - // Setup - when(mockLifecycle.started()).thenReturn(true); - when(mockRepository.createServerSideEncryptedBlobStore()).thenReturn(mockServerSideEncryptedBlobStore); - when(mockRepository.createClientSideEncryptedBlobStore()).thenReturn(mockClientSideEncryptedBlobStore); - - // Test - use both server-side and client-side encryption - BlobStore serverSideResult = provider.blobStore(true); - BlobStore clientSideResult = provider.blobStore(false); - - // Verify - assertEquals(mockServerSideEncryptedBlobStore, serverSideResult); - assertEquals(mockClientSideEncryptedBlobStore, clientSideResult); - assertNotSame(serverSideResult, clientSideResult); - - verify(mockRepository).createServerSideEncryptedBlobStore(); - verify(mockRepository).createClientSideEncryptedBlobStore(); - } - - public void testGetBlobStoreReturnsNullWhenNotInitialized() { - // Test - getBlobStore when server-side encrypted store is not initialized - BlobStore result = provider.getBlobStore(true); - - // Verify - should return null since SetOnce.get() returns null when not set - assertNull(result); - } -} diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index a0df8ecd4bbef..36f4e77228325 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -102,6 +102,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.SourceToParse; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.replication.TestReplicationSource; import org.opensearch.index.seqno.ReplicationTracker; @@ -743,7 +744,7 @@ protected IndexShard newShard( settings.getRemoteStoreTranslogRepository(), new RemoteTranslogTransferTracker(shardRouting.shardId(), 20), DefaultRemoteStoreSettings.INSTANCE, - settings.isServerSideEncryptionEnabled() + RemoteStoreUtils.isServerSideEncryptionEnabledIndex(settings.getIndexMetadata()) ); } return new InternalTranslogFactory();