From 33dc43986dd5f2d30547fdd5e2ea14c985efe3fa Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Fri, 2 May 2025 21:12:13 +1000 Subject: [PATCH 01/24] WIP --- .../s3/RepositoryCredentialsTests.java | 18 +- .../s3/S3PerProjectClientManager.java | 264 ++++++++++++++++++ .../repositories/s3/S3RepositoryPlugin.java | 14 +- .../repositories/s3/S3Service.java | 100 ++++++- .../s3/AwsS3ServiceImplTests.java | 7 +- .../s3/S3BlobContainerRetriesTests.java | 17 +- .../s3/S3ClientSettingsTests.java | 7 +- .../s3/S3PerProjectClientManagerTests.java | 161 +++++++++++ .../repositories/s3/S3RepositoryTests.java | 35 ++- .../repositories/s3/S3ServiceTests.java | 89 ++++-- .../test/ClusterServiceUtils.java | 9 + 11 files changed, 676 insertions(+), 45 deletions(-) create mode 100644 modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java create mode 100644 modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java diff --git a/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java b/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java index 6074219823da1..cfd71ffd5405c 100644 --- a/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java +++ b/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java @@ -17,6 +17,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.cluster.project.ProjectResolver; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; @@ -256,7 +258,12 @@ public ProxyS3RepositoryPlugin(Settings settings) { } @Override - S3Service s3Service(Environment environment, Settings nodeSettings, ResourceWatcherService resourceWatcherService) { + S3Service s3Service( + Environment environment, + ClusterService clusterService, + ProjectResolver projectResolver, + ResourceWatcherService resourceWatcherService + ) { return new ProxyS3Service(environment, nodeSettings, resourceWatcherService); } @@ -293,8 +300,13 @@ public static final class ProxyS3Service extends S3Service { private static final Logger logger = LogManager.getLogger(ProxyS3Service.class); - ProxyS3Service(Environment environment, Settings nodeSettings, ResourceWatcherService resourceWatcherService) { - super(environment, nodeSettings, resourceWatcherService, () -> null); + ProxyS3Service( + Environment environment, + ClusterService clusterService, + ProjectResolver projectResolver, + ResourceWatcherService resourceWatcherService + ) { + super(environment, clusterService, projectResolver, resourceWatcherService, () -> null); } @Override diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java new file mode 100644 index 0000000000000..00e885aaafc65 --- /dev/null +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -0,0 +1,264 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.ProjectId; +import org.elasticsearch.cluster.metadata.ProjectMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.common.settings.ProjectSecrets; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class S3PerProjectClientManager implements ClusterStateListener { + + private static final Logger logger = LogManager.getLogger(S3PerProjectClientManager.class); + + private final Settings settings; + private final Function clientBuilder; + private final Executor executor; + // A map of projectId to clients holder. Adding to and removing from the map happen only in the cluster state listener thread. + private final Map projectClientsHolders; + // Listener for tracking ongoing async closing of obsolete clients. Updated only in the cluster state listener thread. + private volatile SubscribableListener clientsCloseListener = null; + + S3PerProjectClientManager(Settings settings, Function clientBuilder, Executor executor) { + this.settings = settings; + this.clientBuilder = clientBuilder; + this.executor = executor; + this.projectClientsHolders = new ConcurrentHashMap<>(); + } + + // visible for tests + Map getProjectClientsHolders() { + return Map.copyOf(projectClientsHolders); + } + + public void clusterChanged(ClusterChangedEvent event) { + final Map currentProjects = event.state().metadata().projects(); + + final var updatedPerProjectClients = new HashMap(); + for (var project : currentProjects.values()) { + final ProjectSecrets projectSecrets = project.custom(ProjectSecrets.TYPE); + if (projectSecrets == null) { + // This can only happen when a node restarts, it will be processed again when file settings are loaded + continue; + } + final Settings currentSettings = Settings.builder() + // merge with static settings such as max retries etc, exclude secure settings + // TODO: We may need to update this if per-project settings decide to support hierarchical overrides + .put(settings, false) // do not fallback to cluster scoped secrets + .setSecureSettings(projectSecrets.getSettings()) + .build(); + final Map clientSettings = S3ClientSettings.load(currentSettings) + .entrySet() + .stream() + // Skip project clients that have no credentials configured. This should not happen in serverless since all clients should + // have credentials configured. But it is safer to skip them. + .filter(entry -> entry.getValue().credentials != null) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + + // TODO: clientSettings should not be empty, i.e. there should be at least one client configured + // Maybe log a warning if it is empty and continue. The project will not have usable client but that is probably ok. + + // TODO: Building and comparing the whole S3ClientSettings may be inefficient, we could just compare the relevant secrets + if (newOrUpdated(project.id(), clientSettings)) { + updatedPerProjectClients.put(project.id(), new ClientsHolder(clientSettings)); + } + } + + final List clientsHoldersToClose = new ArrayList<>(); + // Updated projects + for (var projectId : updatedPerProjectClients.keySet()) { + final var old = projectClientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); + if (old != null) { + clientsHoldersToClose.add(old); + } + } + // removed projects + for (var projectId : projectClientsHolders.keySet()) { + if (currentProjects.containsKey(projectId) == false) { + final var removed = projectClientsHolders.remove(projectId); + assert removed != null; + clientsHoldersToClose.add(removed); + } + } + // Close stale clients asynchronously without blocking the cluster state thread + if (clientsHoldersToClose.isEmpty() == false) { + final var currentClientsCloseListener = new SubscribableListener(); + final var previousClientsCloseListener = clientsCloseListener; + clientsCloseListener = currentClientsCloseListener; + if (previousClientsCloseListener != null && previousClientsCloseListener.isDone() == false) { + previousClientsCloseListener.addListener( + ActionListener.running(() -> closeClientsAsync(clientsHoldersToClose, currentClientsCloseListener)) + ); + } else { + closeClientsAsync(clientsHoldersToClose, currentClientsCloseListener); + } + } + } + + private void closeClientsAsync(List clientsHoldersToClose, ActionListener listener) { + executor.execute(() -> { + IOUtils.closeWhileHandlingException(clientsHoldersToClose); + listener.onResponse(null); + }); + } + + public AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetadata) { + final var clientsHolder = projectClientsHolders.get(projectId); + if (clientsHolder == null) { + throw new IllegalArgumentException("project [" + projectId + "] does not exist"); + } + final String clientName = S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); + return clientsHolder.client(clientName); + } + + /** + * Similar to S3Service#releaseCachedClients but only clears the cache for the given project. + * All clients for the project are closed and will be recreated on next access, also similar to S3Service#releaseCachedClients + */ + public void releaseProjectClients(ProjectId projectId) { + final var old = projectClientsHolders.get(projectId); + if (old != null) { + old.clearCache(); + } + } + + /** + * Shutdown the manager by closing all clients holders. This is called when the node is shutting down. + * It attempts to wait (1 min) for any async client closing to complete. + */ + public void close() { + IOUtils.closeWhileHandlingException(projectClientsHolders.values()); + final var currentClientsCloseListener = clientsCloseListener; + if (currentClientsCloseListener != null && currentClientsCloseListener.isDone() == false) { + // Wait for async clients closing to be completed + final CountDownLatch latch = new CountDownLatch(1); + currentClientsCloseListener.addListener(ActionListener.running(latch::countDown)); + try { + if (latch.await(1, TimeUnit.MINUTES) == false) { + logger.warn("async closing of s3 clients timed out"); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + // visible for tests + @Nullable + SubscribableListener getClientsCloseListener() { + return clientsCloseListener; + } + + private boolean newOrUpdated(ProjectId projectId, Map currentClientSettings) { + final var old = projectClientsHolders.get(projectId); + if (old == null) { + return true; + } + return currentClientSettings.equals(old.clientSettings()) == false; + } + + /** + * Holder class of s3 clients for a single project. It is instantiated in the cluster state thread with client + * settings. The clients are created and cached lazily when the {@link #client(String)} method is called. + * Cached clients are closed and cleared out when the {@link #clearCache()} method is called. Subsequent calls to + * {@link #client(String)} will recreate them. The call to {@link #close()} method clears the cache as well but + * also flags the holder to be closed so that no new clients can be created. + */ + final class ClientsHolder implements Closeable { + private final AtomicBoolean closed = new AtomicBoolean(false); + private final Map clientSettings; + // Client name -> client reference + private volatile Map clientsCache = Collections.emptyMap(); + + ClientsHolder(Map clientSettings) { + this.clientSettings = clientSettings; + } + + Map clientSettings() { + return clientSettings; + } + + AmazonS3Reference client(String clientName) { + final var clientReference = clientsCache.get(clientName); + // It is ok to retrieve an existing client when the cache is being cleared or the holder is closing. + // As long as there are paired incRef/decRef calls, the client will be closed when the last reference is released + // by either the caller of this method or the clearCache() method. + if (clientReference != null && clientReference.tryIncRef()) { + return clientReference; + } + final var settings = clientSettings.get(clientName); + if (settings == null) { + throw new IllegalArgumentException("client [" + clientName + "] does not exist"); + } + synchronized (this) { + final var existing = clientsCache.get(clientName); + if (existing != null && existing.tryIncRef()) { + return existing; + } + if (closed.get()) { + // Not adding a new client once the manager is closed since there won't be anything to close it + throw new IllegalStateException("client manager is closed"); + } + // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. + // The clearCache() will clear the newly added client. + final var newClientReference = clientBuilder.apply(settings); + clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientName, newClientReference); + return newClientReference; + } + } + + /** + * Clear the cache by closing and clear out all clients. Subsequent {@link #client(String)} calls will recreate + * the clients and populate the cache again. + */ + synchronized void clearCache() { + IOUtils.closeWhileHandlingException(clientsCache.values()); + clientsCache = Collections.emptyMap(); + } + + /** + * Similar to {@link #clearCache()} but also flag the holder to be closed so that no new client can be created. + */ + public void close() { + if (closed.compareAndSet(false, true)) { + clearCache(); + } + } + + // visible for tests + boolean isClosed() { + return closed.get(); + } + } +} diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java index 43520bb123647..e4a2c55f8c3cd 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java @@ -15,6 +15,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.SpecialPermission; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.ProjectResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -87,13 +88,20 @@ protected S3Repository createRepository( @Override public Collection createComponents(PluginServices services) { - service.set(s3Service(services.environment(), services.clusterService().getSettings(), services.resourceWatcherService())); + service.set( + s3Service(services.environment(), services.clusterService(), services.projectResolver(), services.resourceWatcherService()) + ); this.service.get().refreshAndClearCache(S3ClientSettings.load(settings)); return List.of(service.get()); } - S3Service s3Service(Environment environment, Settings nodeSettings, ResourceWatcherService resourceWatcherService) { - return new S3Service(environment, nodeSettings, resourceWatcherService, S3RepositoryPlugin::getDefaultRegion); + S3Service s3Service( + Environment environment, + ClusterService clusterService, + ProjectResolver projectResolver, + ResourceWatcherService resourceWatcherService + ) { + return new S3Service(environment, clusterService, projectResolver, resourceWatcherService, S3RepositoryPlugin::getDefaultRegion); } private static Region getDefaultRegion() { diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 82f0ea5964963..4ef721e95b45d 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -39,8 +39,11 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.coordination.stateless.StoreHeartbeatService; +import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.project.ProjectResolver; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Setting; @@ -124,13 +127,17 @@ class S3Service extends AbstractLifecycleComponent { final TimeValue compareAndExchangeTimeToLive; final TimeValue compareAndExchangeAntiContentionDelay; final boolean isStateless; + @Nullable // if multi-project is disabled + private final S3PerProjectClientManager s3PerProjectClientManager; S3Service( Environment environment, - Settings nodeSettings, + ClusterService clusterService, + ProjectResolver projectResolver, ResourceWatcherService resourceWatcherService, Supplier defaultRegionSupplier ) { + final Settings settings = clusterService.getSettings(); webIdentityTokenCredentialsProvider = new CustomWebIdentityTokenCredentialsProvider( environment, System::getenv, @@ -138,10 +145,26 @@ class S3Service extends AbstractLifecycleComponent { Clock.systemUTC(), resourceWatcherService ); - compareAndExchangeTimeToLive = REPOSITORY_S3_CAS_TTL_SETTING.get(nodeSettings); - compareAndExchangeAntiContentionDelay = REPOSITORY_S3_CAS_ANTI_CONTENTION_DELAY_SETTING.get(nodeSettings); - isStateless = DiscoveryNode.isStateless(nodeSettings); + compareAndExchangeTimeToLive = REPOSITORY_S3_CAS_TTL_SETTING.get(settings); + compareAndExchangeAntiContentionDelay = REPOSITORY_S3_CAS_ANTI_CONTENTION_DELAY_SETTING.get(settings); + isStateless = DiscoveryNode.isStateless(settings); defaultRegionSetter = new RunOnce(() -> defaultRegion = defaultRegionSupplier.get()); + if (projectResolver.supportsMultipleProjects()) { + s3PerProjectClientManager = new S3PerProjectClientManager( + settings, + this::buildClientReference, + clusterService.threadPool().generic() + ); + clusterService.addListener(s3PerProjectClientManager); + } else { + s3PerProjectClientManager = null; + } + } + + // visible to tests + @Nullable + S3PerProjectClientManager getS3PerProjectClientManager() { + return s3PerProjectClientManager; } /** @@ -177,17 +200,42 @@ public AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { if (existing != null && existing.tryIncRef()) { return existing; } - final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); - Releasable toRelease = httpClient::close; - try { - final AmazonS3Reference clientReference = new AmazonS3Reference(buildClient(clientSettings, httpClient), httpClient); - clientReference.mustIncRef(); - clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientSettings, clientReference); - toRelease = null; - return clientReference; - } finally { - Releasables.close(toRelease); - } + final AmazonS3Reference clientReference = buildClientReference(clientSettings); + clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientSettings, clientReference); + return clientReference; + } + } + + /** + * Attempts to retrieve a project client from the project client manager. Throws if project-id or the client name does not exist. + * THe client maybe initialized lazily. + * Delegates to {@link #client(RepositoryMetadata)} when either of the followings is true: + * 1. Per-project client is disabled + * 2. Blobstore is cluster level (projectId = null) + */ + public AmazonS3Reference client(@Nullable ProjectId projectId, RepositoryMetadata repositoryMetadata) { + if (s3PerProjectClientManager == null) { + // Multi-Project is disabled and we have a single default project + assert ProjectId.DEFAULT.equals(projectId) : projectId; + return client(repositoryMetadata); + } else if (projectId == null) { + // Multi-Project is enabled and we are retrieving a client for the cluster level blobstore + return client(repositoryMetadata); + } else { + return s3PerProjectClientManager.client(projectId, repositoryMetadata); + } + } + + private AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { + final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); + Releasable toRelease = httpClient::close; + try { + final AmazonS3Reference clientReference = new AmazonS3Reference(buildClient(clientSettings, httpClient), httpClient); + clientReference.mustIncRef(); + toRelease = null; + return clientReference; + } finally { + Releasables.close(toRelease); } } @@ -457,6 +505,25 @@ public void onBlobStoreClose() { releaseCachedClients(); } + /** + * Release all project clients. + * Delegates to {@link #onBlobStoreClose()} when either of the followings is true: + * 1. Per-project client is disabled + * 2. Blobstore is cluster level (projectId = null) + */ + public void onBlobStoreClose(@Nullable ProjectId projectId) { + if (s3PerProjectClientManager == null) { + // Multi-Project is disabled and we have a single default project + assert ProjectId.DEFAULT.equals(projectId) : projectId; + onBlobStoreClose(); + } else if (projectId == null) { + // Multi-Project is enabled and this is for the cluster level blobstore + onBlobStoreClose(); + } else { + s3PerProjectClientManager.releaseProjectClients(projectId); + } + } + @Override protected void doStart() { defaultRegionSetter.run(); @@ -468,6 +535,9 @@ protected void doStop() {} @Override public void doClose() throws IOException { releaseCachedClients(); + if (s3PerProjectClientManager != null) { + s3PerProjectClientManager.close(); + } webIdentityTokenCredentialsProvider.close(); } diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java index 5244c956df75d..895d24826beda 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java @@ -22,10 +22,13 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -238,9 +241,11 @@ private void assertEndpoint(Settings repositorySettings, Settings settings, Stri public void testEndPointAndRegionOverrides() throws IOException { try ( + TestThreadPool threadPool = new TestThreadPool(getTestName()); S3Service s3Service = new S3Service( mock(Environment.class), - Settings.EMPTY, + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of("es-test-region") ) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 1fb929e654fc4..28809b05f68b1 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Level; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.BackoffPolicy; import org.elasticsearch.common.Strings; import org.elasticsearch.common.blobstore.BlobContainer; @@ -54,8 +55,10 @@ import org.elasticsearch.telemetry.InstrumentType; import org.elasticsearch.telemetry.Measurement; import org.elasticsearch.telemetry.RecordingMeterRegistry; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.hamcrest.Matcher; import org.junit.After; @@ -119,6 +122,7 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTestCase { private static final int MAX_NUMBER_SNAPSHOT_DELETE_RETRIES = 10; + private TestThreadPool threadPool; private S3Service service; private volatile boolean shouldErrorOnDns; private RecordingMeterRegistry recordingMeterRegistry; @@ -126,7 +130,14 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes @Before public void setUp() throws Exception { shouldErrorOnDns = false; - service = new S3Service(Mockito.mock(Environment.class), Settings.EMPTY, Mockito.mock(ResourceWatcherService.class), () -> null) { + threadPool = new TestThreadPool(getTestClass().getName()); + service = new S3Service( + Mockito.mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + Mockito.mock(ResourceWatcherService.class), + () -> null + ) { private InetAddress[] resolveHost(String host) throws UnknownHostException { assertEquals("127.0.0.1", host); if (shouldErrorOnDns && randomBoolean() && randomBoolean()) { @@ -161,6 +172,7 @@ Optional getConnectionAcquisitionTimeout() { @After public void tearDown() throws Exception { IOUtils.close(service); + threadPool.close(); super.tearDown(); } @@ -1307,7 +1319,8 @@ public void testRetryOn403InStateless() { service = new S3Service( Mockito.mock(Environment.class), - Settings.builder().put(STATELESS_ENABLED_SETTING_NAME, "true").build(), + ClusterServiceUtils.createClusterService(threadPool, Settings.builder().put(STATELESS_ENABLED_SETTING_NAME, "true").build()), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, Mockito.mock(ResourceWatcherService.class), () -> null ); diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java index 5e7f083de8eb2..2f161f1576195 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java @@ -13,10 +13,13 @@ import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import software.amazon.awssdk.regions.Region; +import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.mockito.Mockito; @@ -182,9 +185,11 @@ public void testRegionCanBeSet() { assertThat(settings.get("other").region, is(randomRegion)); try ( + TestThreadPool threadPool = new TestThreadPool(getTestName()); var s3Service = new S3Service( Mockito.mock(Environment.class), - Settings.EMPTY, + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, Mockito.mock(ResourceWatcherService.class), () -> null ) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java new file mode 100644 index 0000000000000..0a25eb75fc68a --- /dev/null +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java @@ -0,0 +1,161 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.regions.Region; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.ProjectId; +import org.elasticsearch.cluster.metadata.ProjectMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.TestProjectResolvers; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.ProjectSecrets; +import org.elasticsearch.common.settings.SecureClusterStateSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ClusterServiceUtils; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.watcher.ResourceWatcherService; + +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Set; + +import static org.hamcrest.Matchers.anEmptyMap; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.mock; + +public class S3PerProjectClientManagerTests extends ESTestCase { + + private TestThreadPool threadPool; + private ClusterService clusterService; + private S3Service s3Service; + + @Override + public void setUp() throws Exception { + super.setUp(); + final var mockSecureSettings = new MockSecureSettings(); + mockSecureSettings.setString("s3.client.default.access_key", "cluster_access_key"); + mockSecureSettings.setString("s3.client.default.secret_key", "cluster_secret_key"); + final Settings settings = Settings.builder() + .put("s3.client.default.max_retries", 9) + .put("s3.client.backup.read_timeout", "99s") + .setSecureSettings(mockSecureSettings) + .build(); + threadPool = new TestThreadPool(getTestName()); + clusterService = ClusterServiceUtils.createClusterService(threadPool, settings); + s3Service = new S3Service( + mock(Environment.class), + clusterService, + TestProjectResolvers.allProjects(), + mock(ResourceWatcherService.class), + () -> Region.of("es-test-region") + ); + s3Service.start(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + s3Service.close(); + clusterService.close(); + threadPool.close(); + final var s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); + final var clientsCloseListener = s3PerProjectClientManager.getClientsCloseListener(); + assertTrue(clientsCloseListener == null || clientsCloseListener.isDone()); + s3PerProjectClientManager.getProjectClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); + } + + public void testBasic() { + final var s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); + assertNotNull(s3PerProjectClientManager); + assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + + final ProjectId projectId = randomUniqueProjectId(); + final var repositoryMetadata = createRepositoryMetadata("default"); + { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> s3PerProjectClientManager.client(projectId, repositoryMetadata) + ); + assertThat(e.getMessage(), containsString("project [" + projectId + "] does not exist")); + } + + final var mockSecureSettings = new MockSecureSettings(); + mockSecureSettings.setFile("s3.client.default.access_key", (projectId + "_access_key").getBytes(StandardCharsets.UTF_8)); + mockSecureSettings.setFile("s3.client.default.secret_key", (projectId + "_secret_key").getBytes(StandardCharsets.UTF_8)); + final ClusterState stateWithProject = ClusterState.builder(clusterService.state()) + .putProjectMetadata( + ProjectMetadata.builder(projectId) + .putCustom(ProjectSecrets.TYPE, new ProjectSecrets(new SecureClusterStateSettings(mockSecureSettings))) + ) + .build(); + ClusterServiceUtils.setState(clusterService, stateWithProject); + { + final var clientsHolder = s3PerProjectClientManager.getProjectClientsHolders().get(projectId); + assertNotNull(clientsHolder); + final Map s3ClientSettingsMap = clientsHolder.clientSettings(); + assertThat(s3ClientSettingsMap.keySet(), equalTo(Set.of("default"))); + final S3ClientSettings clientSettings = s3ClientSettingsMap.get("default"); + // Picks up the correct project scoped credentials + assertThat( + clientSettings.credentials, + equalTo(AwsBasicCredentials.create(projectId + "_access_key", projectId + "_secret_key")) + ); + // Inherit setting override from the cluster client of the same name + assertThat(clientSettings.maxRetries, equalTo(9)); + // Does not inherit setting override from a cluster client of a different name + assertThat((long) clientSettings.readTimeoutMillis, equalTo(S3ClientSettings.Defaults.READ_TIMEOUT.millis())); + + // Retrieve client for the 1st time + final AmazonS3Reference initialClient = s3PerProjectClientManager.client(projectId, repositoryMetadata); + // Client is cached when retrieved again + assertThat(initialClient, sameInstance(s3PerProjectClientManager.client(projectId, repositoryMetadata))); + + // Client should be released and recreated again on access + s3PerProjectClientManager.releaseProjectClients(projectId); + final AmazonS3Reference clientAgain = s3PerProjectClientManager.client(projectId, repositoryMetadata); + assertThat(clientAgain, not(sameInstance(initialClient))); + clientAgain.decRef(); + + // Release the initial client and all references should be cleared + initialClient.decRef(); + initialClient.decRef(); + assertFalse(initialClient.hasReferences()); + + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> s3PerProjectClientManager.client(projectId, createRepositoryMetadata("backup")) + ); + assertThat(e.getMessage(), containsString("client [backup] does not exist")); + } + + { + final ProjectId anotherProjectId = randomUniqueProjectId(); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> s3PerProjectClientManager.client(anotherProjectId, repositoryMetadata) + ); + assertThat(e.getMessage(), containsString("project [" + anotherProjectId + "] does not exist")); + } + } + + private static RepositoryMetadata createRepositoryMetadata(String clientName) { + return new RepositoryMetadata(randomIdentifier(), "s3", Settings.builder().put("client", clientName).build()); + } + +} diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java index 51362b9435489..e19536e0df6d7 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java @@ -13,6 +13,9 @@ import software.amazon.awssdk.services.s3.S3Client; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.ProjectResolver; +import org.elasticsearch.cluster.project.TestProjectResolvers; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.ReferenceDocs; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -23,7 +26,9 @@ import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.hamcrest.Matchers; @@ -39,6 +44,20 @@ public class S3RepositoryTests extends ESTestCase { + private TestThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getTestName()); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.close(); + } + private static class DummyS3Client implements S3Client { @Override @@ -54,8 +73,13 @@ public String serviceName() { private static class DummyS3Service extends S3Service { - DummyS3Service(Environment environment, ResourceWatcherService resourceWatcherService) { - super(environment, Settings.EMPTY, resourceWatcherService, () -> null); + DummyS3Service( + Environment environment, + ClusterService clusterService, + ProjectResolver projectResolver, + ResourceWatcherService resourceWatcherService + ) { + super(environment, clusterService, projectResolver, resourceWatcherService, () -> null); } @Override @@ -155,7 +179,12 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { return new S3Repository( metadata, NamedXContentRegistry.EMPTY, - new DummyS3Service(mock(Environment.class), mock(ResourceWatcherService.class)), + new DummyS3Service( + mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class) + ), BlobStoreTestUtil.mockClusterService(), MockBigArrays.NON_RECYCLING_INSTANCE, new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java index dd18932cea7d3..778a55098bf20 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java @@ -20,13 +20,16 @@ import org.apache.logging.log4j.Level; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import java.io.IOException; @@ -38,14 +41,30 @@ public class S3ServiceTests extends ESTestCase { + private TestThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getTestName()); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.close(); + } + public void testCachedClientsAreReleased() throws IOException { final S3Service s3Service = new S3Service( mock(Environment.class), - Settings.EMPTY, + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of("es-test-region") ); s3Service.start(); + assertNull(s3Service.getS3PerProjectClientManager()); final Settings settings = Settings.builder().put("endpoint", "http://first").build(); final RepositoryMetadata metadata1 = new RepositoryMetadata("first", "s3", settings); final RepositoryMetadata metadata2 = new RepositoryMetadata("second", "s3", settings); @@ -94,11 +113,20 @@ public void testRetryOn403RetryPolicy() { @TestLogging(reason = "testing WARN log output", value = "org.elasticsearch.repositories.s3.S3Service:WARN") public void testGetClientRegionFromSetting() { final var regionRequested = new AtomicBoolean(); - try (var s3Service = new S3Service(mock(Environment.class), Settings.EMPTY, mock(ResourceWatcherService.class), () -> { - assertTrue(regionRequested.compareAndSet(false, true)); - return randomFrom(randomFrom(Region.regions()), Region.of(randomIdentifier()), null); - })) { + try ( + var s3Service = new S3Service( + mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class), + () -> { + assertTrue(regionRequested.compareAndSet(false, true)); + return randomFrom(randomFrom(Region.regions()), Region.of(randomIdentifier()), null); + } + ) + ) { s3Service.start(); + assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -124,11 +152,20 @@ public void testGetClientRegionFromSetting() { @TestLogging(reason = "testing WARN log output", value = "org.elasticsearch.repositories.s3.S3Service:WARN") public void testGetClientRegionFromEndpointSettingGuess() { final var regionRequested = new AtomicBoolean(); - try (var s3Service = new S3Service(mock(Environment.class), Settings.EMPTY, mock(ResourceWatcherService.class), () -> { - assertTrue(regionRequested.compareAndSet(false, true)); - return randomFrom(randomFrom(Region.regions()), Region.of(randomIdentifier()), null); - })) { + try ( + var s3Service = new S3Service( + mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class), + () -> { + assertTrue(regionRequested.compareAndSet(false, true)); + return randomFrom(randomFrom(Region.regions()), Region.of(randomIdentifier()), null); + } + ) + ) { s3Service.start(); + assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -174,11 +211,20 @@ public void testGetClientRegionFromEndpointSettingGuess() { public void testGetClientRegionFromDefault() { final var regionRequested = new AtomicBoolean(); final var defaultRegion = randomBoolean() ? randomFrom(Region.regions()) : Region.of(randomIdentifier()); - try (var s3Service = new S3Service(mock(Environment.class), Settings.EMPTY, mock(ResourceWatcherService.class), () -> { - assertTrue(regionRequested.compareAndSet(false, true)); - return defaultRegion; - })) { + try ( + var s3Service = new S3Service( + mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class), + () -> { + assertTrue(regionRequested.compareAndSet(false, true)); + return defaultRegion; + } + ) + ) { s3Service.start(); + assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -201,11 +247,20 @@ public void testGetClientRegionFromDefault() { @TestLogging(reason = "testing WARN log output", value = "org.elasticsearch.repositories.s3.S3Service:WARN") public void testGetClientRegionFallbackToUsEast1() { final var regionRequested = new AtomicBoolean(); - try (var s3Service = new S3Service(mock(Environment.class), Settings.EMPTY, mock(ResourceWatcherService.class), () -> { - assertTrue(regionRequested.compareAndSet(false, true)); - return null; - })) { + try ( + var s3Service = new S3Service( + mock(Environment.class), + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class), + () -> { + assertTrue(regionRequested.compareAndSet(false, true)); + return null; + } + ) + ) { s3Service.start(); + assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java index becb6aaa7de19..e0e7505191da9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java @@ -124,6 +124,15 @@ public static ClusterService createClusterService(ThreadPool threadPool, Discove return createClusterService(threadPool, localNode, Settings.EMPTY, clusterSettings); } + public static ClusterService createClusterService(ThreadPool threadPool, Settings providedSettings) { + return createClusterService( + threadPool, + DiscoveryNodeUtils.create("node", "node"), + providedSettings, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + } + public static ClusterService createClusterService( ThreadPool threadPool, DiscoveryNode localNode, From a8dc2787cbc5f8d55288afc66d89fb706336642b Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Sat, 3 May 2025 18:31:17 +1000 Subject: [PATCH 02/24] add more tests --- .../s3/RepositoryCredentialsTests.java | 2 +- .../s3/S3PerProjectClientManager.java | 56 ++- .../repositories/s3/S3Service.java | 5 +- .../s3/S3PerProjectClientManagerTests.java | 461 +++++++++++++++--- .../common/settings/ProjectSecrets.java | 5 + 5 files changed, 445 insertions(+), 84 deletions(-) diff --git a/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java b/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java index cfd71ffd5405c..420794ed75d75 100644 --- a/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java +++ b/modules/repository-s3/qa/insecure-credentials/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java @@ -264,7 +264,7 @@ S3Service s3Service( ProjectResolver projectResolver, ResourceWatcherService resourceWatcherService ) { - return new ProxyS3Service(environment, nodeSettings, resourceWatcherService); + return new ProxyS3Service(environment, clusterService, projectResolver, resourceWatcherService); } /** diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index 00e885aaafc65..b3e248e61fdcb 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -15,7 +15,6 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; -import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.common.settings.ProjectSecrets; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.Maps; @@ -41,8 +40,9 @@ public class S3PerProjectClientManager implements ClusterStateListener { private static final Logger logger = LogManager.getLogger(S3PerProjectClientManager.class); + private static final String S3_SETTING_PREFIX = "s3."; - private final Settings settings; + private final Settings nodeS3Settings; private final Function clientBuilder; private final Executor executor; // A map of projectId to clients holder. Adding to and removing from the map happen only in the cluster state listener thread. @@ -51,7 +51,10 @@ public class S3PerProjectClientManager implements ClusterStateListener { private volatile SubscribableListener clientsCloseListener = null; S3PerProjectClientManager(Settings settings, Function clientBuilder, Executor executor) { - this.settings = settings; + this.nodeS3Settings = Settings.builder() + .put(settings.getByPrefix(S3_SETTING_PREFIX), false) // not use any cluster scoped secrets + .normalizePrefix(S3_SETTING_PREFIX) + .build(); this.clientBuilder = clientBuilder; this.executor = executor; this.projectClientsHolders = new ConcurrentHashMap<>(); @@ -66,36 +69,48 @@ public void clusterChanged(ClusterChangedEvent event) { final Map currentProjects = event.state().metadata().projects(); final var updatedPerProjectClients = new HashMap(); + final List clientsHoldersToClose = new ArrayList<>(); for (var project : currentProjects.values()) { final ProjectSecrets projectSecrets = project.custom(ProjectSecrets.TYPE); - if (projectSecrets == null) { - // This can only happen when a node restarts, it will be processed again when file settings are loaded + // Project secrets can be null when node restarts. It may not have s3 credentials if s3 is not used. + if (projectSecrets == null || projectSecrets.getSettingNames().stream().noneMatch(key -> key.startsWith("s3."))) { + // Most likely there won't be any existing client, but attempt to remove it anyway just in case + final ClientsHolder removed = projectClientsHolders.remove(project.id()); + if (removed != null) { + clientsHoldersToClose.add(removed); + } continue; } + final Settings currentSettings = Settings.builder() - // merge with static settings such as max retries etc, exclude secure settings + // merge with static settings such as max retries etc // TODO: We may need to update this if per-project settings decide to support hierarchical overrides - .put(settings, false) // do not fallback to cluster scoped secrets + .put(nodeS3Settings) .setSecureSettings(projectSecrets.getSettings()) .build(); final Map clientSettings = S3ClientSettings.load(currentSettings) .entrySet() .stream() - // Skip project clients that have no credentials configured. This should not happen in serverless since all clients should - // have credentials configured. But it is safer to skip them. + // Skip project clients that have no credentials configured. This should not happen in serverless. + // But it is safer to skip them and is also a more consistent behaviour with the cases when + // project secrets are not present. .filter(entry -> entry.getValue().credentials != null) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); - // TODO: clientSettings should not be empty, i.e. there should be at least one client configured - // Maybe log a warning if it is empty and continue. The project will not have usable client but that is probably ok. + if (clientSettings.isEmpty()) { + // clientSettings should not be empty, i.e. there should be at least one client configured. + // But if it does somehow happen, log a warning and continue. The project will not have usable client but that is ok. + logger.warn("Skipping project [{}] with no client settings", project.id()); + continue; + } - // TODO: Building and comparing the whole S3ClientSettings may be inefficient, we could just compare the relevant secrets + // TODO: If performance is an issue, we may consider comparing just the relevant project secrets for new or updated clients + // and avoid building the clientSettings if (newOrUpdated(project.id(), clientSettings)) { - updatedPerProjectClients.put(project.id(), new ClientsHolder(clientSettings)); + updatedPerProjectClients.put(project.id(), new ClientsHolder(project.id(), clientSettings)); } } - final List clientsHoldersToClose = new ArrayList<>(); // Updated projects for (var projectId : updatedPerProjectClients.keySet()) { final var old = projectClientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); @@ -133,12 +148,12 @@ private void closeClientsAsync(List clientsHoldersToClose, Action }); } - public AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetadata) { + public AmazonS3Reference client(ProjectId projectId, String clientName) { + assert projectId != null && ProjectId.DEFAULT.equals(projectId) == false : projectId; final var clientsHolder = projectClientsHolders.get(projectId); if (clientsHolder == null) { - throw new IllegalArgumentException("project [" + projectId + "] does not exist"); + throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); } - final String clientName = S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); return clientsHolder.client(clientName); } @@ -147,6 +162,7 @@ public AmazonS3Reference client(ProjectId projectId, RepositoryMetadata reposito * All clients for the project are closed and will be recreated on next access, also similar to S3Service#releaseCachedClients */ public void releaseProjectClients(ProjectId projectId) { + assert projectId != null && ProjectId.DEFAULT.equals(projectId) == false : projectId; final var old = projectClientsHolders.get(projectId); if (old != null) { old.clearCache(); @@ -197,11 +213,13 @@ private boolean newOrUpdated(ProjectId projectId, Map */ final class ClientsHolder implements Closeable { private final AtomicBoolean closed = new AtomicBoolean(false); + private final ProjectId projectId; private final Map clientSettings; // Client name -> client reference private volatile Map clientsCache = Collections.emptyMap(); - ClientsHolder(Map clientSettings) { + ClientsHolder(ProjectId projectId, Map clientSettings) { + this.projectId = projectId; this.clientSettings = clientSettings; } @@ -219,7 +237,7 @@ AmazonS3Reference client(String clientName) { } final var settings = clientSettings.get(clientName); if (settings == null) { - throw new IllegalArgumentException("client [" + clientName + "] does not exist"); + throw new IllegalArgumentException("s3 client [" + clientName + "] does not exist for project [" + projectId + "]"); } synchronized (this) { final var existing = clientsCache.get(clientName); diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 4ef721e95b45d..a1f3cdc78d78e 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -222,11 +222,12 @@ public AmazonS3Reference client(@Nullable ProjectId projectId, RepositoryMetadat // Multi-Project is enabled and we are retrieving a client for the cluster level blobstore return client(repositoryMetadata); } else { - return s3PerProjectClientManager.client(projectId, repositoryMetadata); + final String clientName = S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); + return s3PerProjectClientManager.client(projectId, clientName); } } - private AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { + protected AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); Releasable toRelease = httpClient::close; try { diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java index 0a25eb75fc68a..eff51187f784e 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java @@ -9,19 +9,27 @@ package org.elasticsearch.repositories.s3; +import joptsimple.internal.Strings; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.http.ExecutableHttpRequest; +import software.amazon.awssdk.http.HttpExecuteRequest; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; import software.amazon.awssdk.regions.Region; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.project.TestProjectResolvers; +import org.elasticsearch.cluster.routing.GlobalRoutingTable; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.ProjectSecrets; import org.elasticsearch.common.settings.SecureClusterStateSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; @@ -29,33 +37,68 @@ import org.elasticsearch.watcher.ResourceWatcherService; import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.hamcrest.Matchers.anEmptyMap; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Mockito.mock; public class S3PerProjectClientManagerTests extends ESTestCase { + private Map s3SecretsIdGenerators; + private List clientNames; + private Map clusterClientsSettings; private TestThreadPool threadPool; private ClusterService clusterService; private S3Service s3Service; + private S3PerProjectClientManager s3PerProjectClientManager; + private final AtomicReference clientRefsCloseLatchRef = new AtomicReference<>(); + private final AtomicBoolean closeInternalInvoked = new AtomicBoolean(false); @Override public void setUp() throws Exception { super.setUp(); + s3SecretsIdGenerators = ConcurrentCollections.newConcurrentMap(); + clientRefsCloseLatchRef.set(null); + closeInternalInvoked.set(false); + clientNames = IntStream.range(0, between(2, 5)).mapToObj(i -> randomIdentifier() + "_" + i).toList(); + + final Settings.Builder builder = Settings.builder(); final var mockSecureSettings = new MockSecureSettings(); - mockSecureSettings.setString("s3.client.default.access_key", "cluster_access_key"); - mockSecureSettings.setString("s3.client.default.secret_key", "cluster_secret_key"); - final Settings settings = Settings.builder() - .put("s3.client.default.max_retries", 9) - .put("s3.client.backup.read_timeout", "99s") - .setSecureSettings(mockSecureSettings) - .build(); + clientNames.forEach(clientName -> { + mockSecureSettings.setString("s3.client." + clientName + ".access_key", clientName + "_cluster_access_key"); + mockSecureSettings.setString("s3.client." + clientName + ".secret_key", clientName + "_cluster_secret_key"); + if (randomBoolean()) { + builder.put("s3.client." + clientName + ".max_retries", between(1, 10)); + } + if (randomBoolean()) { + builder.put("s3.client." + clientName + ".read_timeout", between(1, 99) + "s"); + } + if (randomBoolean()) { + builder.put("s3.client." + clientName + ".max_connections", between(1, 100)); + } + }); + + final Settings settings = builder.setSecureSettings(mockSecureSettings).build(); + clusterClientsSettings = S3ClientSettings.load(settings); threadPool = new TestThreadPool(getTestName()); clusterService = ClusterServiceUtils.createClusterService(threadPool, settings); s3Service = new S3Service( @@ -64,7 +107,31 @@ public void setUp() throws Exception { TestProjectResolvers.allProjects(), mock(ResourceWatcherService.class), () -> Region.of("es-test-region") - ); + ) { + @Override + protected AmazonS3Reference buildClientReference(S3ClientSettings clientSettings) { + final var original = super.buildClientReference(clientSettings); + final var closeLatch = clientRefsCloseLatchRef.get(); + if (closeLatch == null) { + return original; + } + + original.decRef(); + final AmazonS3Reference proxy = new AmazonS3Reference(original.client(), DummySdkHttpClient.INSTANCE) { + @Override + protected void closeInternal() { + closeInternalInvoked.set(true); + safeAwait(closeLatch); + original.close(); + } + }; + proxy.mustIncRef(); + return proxy; + } + }; + s3Service.refreshAndClearCache(S3ClientSettings.load(settings)); + s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); + assertNotNull(s3PerProjectClientManager); s3Service.start(); } @@ -74,88 +141,358 @@ public void tearDown() throws Exception { s3Service.close(); clusterService.close(); threadPool.close(); - final var s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); final var clientsCloseListener = s3PerProjectClientManager.getClientsCloseListener(); assertTrue(clientsCloseListener == null || clientsCloseListener.isDone()); s3PerProjectClientManager.getProjectClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); } - public void testBasic() { - final var s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); - assertNotNull(s3PerProjectClientManager); + public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); - final ProjectId projectId = randomUniqueProjectId(); - final var repositoryMetadata = createRepositoryMetadata("default"); - { - final IllegalArgumentException e = expectThrows( - IllegalArgumentException.class, - () -> s3PerProjectClientManager.client(projectId, repositoryMetadata) - ); - assertThat(e.getMessage(), containsString("project [" + projectId + "] does not exist")); - } + // No project secrets at all + ClusterServiceUtils.setState( + clusterService, + ClusterState.builder(clusterService.state()).putProjectMetadata(ProjectMetadata.builder(projectId)).build() + ); + assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + + // Project secrets but no s3 credentials final var mockSecureSettings = new MockSecureSettings(); - mockSecureSettings.setFile("s3.client.default.access_key", (projectId + "_access_key").getBytes(StandardCharsets.UTF_8)); - mockSecureSettings.setFile("s3.client.default.secret_key", (projectId + "_secret_key").getBytes(StandardCharsets.UTF_8)); - final ClusterState stateWithProject = ClusterState.builder(clusterService.state()) - .putProjectMetadata( - ProjectMetadata.builder(projectId) - .putCustom(ProjectSecrets.TYPE, new ProjectSecrets(new SecureClusterStateSettings(mockSecureSettings))) - ) - .build(); - ClusterServiceUtils.setState(clusterService, stateWithProject); + mockSecureSettings.setFile( + Strings.join(randomList(1, 5, ESTestCase::randomIdentifier), "."), + randomByteArrayOfLength(between(8, 20)) + ); + ClusterServiceUtils.setState( + clusterService, + ClusterState.builder(clusterService.state()) + .putProjectMetadata( + ProjectMetadata.builder(projectId) + .putCustom(ProjectSecrets.TYPE, new ProjectSecrets(new SecureClusterStateSettings(mockSecureSettings))) + ) + .build() + ); + assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + } + + public void testClientsLifeCycleForSingleProject() { + final ProjectId projectId = randomUniqueProjectId(); + final String clientName = randomFrom(clientNames); + final String anotherClientName = randomValueOtherThan(clientName, () -> randomFrom(clientNames)); + + // Configure project secrets for one client + assertClientNotFound(projectId, clientName); + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); { - final var clientsHolder = s3PerProjectClientManager.getProjectClientsHolders().get(projectId); - assertNotNull(clientsHolder); - final Map s3ClientSettingsMap = clientsHolder.clientSettings(); - assertThat(s3ClientSettingsMap.keySet(), equalTo(Set.of("default"))); - final S3ClientSettings clientSettings = s3ClientSettingsMap.get("default"); - // Picks up the correct project scoped credentials - assertThat( - clientSettings.credentials, - equalTo(AwsBasicCredentials.create(projectId + "_access_key", projectId + "_secret_key")) - ); - // Inherit setting override from the cluster client of the same name - assertThat(clientSettings.maxRetries, equalTo(9)); - // Does not inherit setting override from a cluster client of a different name - assertThat((long) clientSettings.readTimeoutMillis, equalTo(S3ClientSettings.Defaults.READ_TIMEOUT.millis())); + assertProjectClientSettings(projectId, clientName); // Retrieve client for the 1st time - final AmazonS3Reference initialClient = s3PerProjectClientManager.client(projectId, repositoryMetadata); + final AmazonS3Reference initialClient = s3PerProjectClientManager.client(projectId, clientName); + assertClientCredentials(projectId, clientName, initialClient); // Client is cached when retrieved again - assertThat(initialClient, sameInstance(s3PerProjectClientManager.client(projectId, repositoryMetadata))); + assertThat(initialClient, sameInstance(s3PerProjectClientManager.client(projectId, clientName))); + + // Client not configured cannot be accessed + assertClientNotFound(projectId, anotherClientName); // Client should be released and recreated again on access s3PerProjectClientManager.releaseProjectClients(projectId); - final AmazonS3Reference clientAgain = s3PerProjectClientManager.client(projectId, repositoryMetadata); - assertThat(clientAgain, not(sameInstance(initialClient))); - clientAgain.decRef(); + final AmazonS3Reference clientUpdated = s3PerProjectClientManager.client(projectId, clientName); + assertThat(clientUpdated, not(sameInstance(initialClient))); + clientUpdated.decRef(); // Release the initial client and all references should be cleared initialClient.decRef(); initialClient.decRef(); assertFalse(initialClient.hasReferences()); - final IllegalArgumentException e = expectThrows( - IllegalArgumentException.class, - () -> s3PerProjectClientManager.client(projectId, createRepositoryMetadata("backup")) + // Update client secrets should release and recreate the client + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName, anotherClientName)); + assertProjectClientSettings(projectId, clientName, anotherClientName); + final AmazonS3Reference clientUpdateAgain = s3PerProjectClientManager.client(projectId, clientName); + assertThat(clientUpdateAgain, not(sameInstance(clientUpdated))); + clientUpdateAgain.decRef(); + + // A different client for a different client name + final AmazonS3Reference antherClient = s3PerProjectClientManager.client(projectId, anotherClientName); + assertClientCredentials(projectId, anotherClientName, antherClient); + assertThat(antherClient, not(sameInstance(clientUpdateAgain))); + antherClient.decRef(); + } + + // Remove project secrets + if (randomBoolean()) { + updateProjectInClusterState(projectId, Map.of()); + } else { + removeProjectFromClusterState(projectId); + } + assertClientNotFound(projectId, clientName); + } + + public void testClientsForMultipleProjects() throws InterruptedException { + final List projectIds = randomList(2, 8, ESTestCase::randomUniqueProjectId); + + final List threads = projectIds.stream().map(projectId -> new Thread(() -> { + final int iterations = between(1, 3); + for (var i = 0; i < iterations; i++) { + final List clientNames = randomNonEmptySubsetOf(this.clientNames); + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientNames.toArray(String[]::new))); + + assertProjectClientSettings(projectId, clientNames.toArray(String[]::new)); + for (var clientName : shuffledList(clientNames)) { + try (var clientRef = s3PerProjectClientManager.client(projectId, clientName)) { + assertClientCredentials(projectId, clientName, clientRef); + } + } + + if (randomBoolean()) { + final Map previousClientRefs = clientNames.stream() + .map(clientName -> Map.entry(clientName, s3PerProjectClientManager.client(projectId, clientName))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + s3PerProjectClientManager.releaseProjectClients(projectId); + previousClientRefs.forEach((clientName, previousClientRef) -> { + final AmazonS3Reference currentClientRef = s3PerProjectClientManager.client(projectId, clientName); + assertThat(currentClientRef, not(sameInstance(previousClientRef))); + assertClientCredentials(projectId, clientName, currentClientRef); + currentClientRef.decRef(); + previousClientRef.decRef(); + }); + } else if (randomBoolean()) { + if (randomBoolean()) { + updateProjectInClusterState(projectId, Map.of()); + } else { + removeProjectFromClusterState(projectId); + } + assertThat(s3PerProjectClientManager.getProjectClientsHolders(), not(hasKey(projectId))); + clientNames.forEach(clientName -> assertClientNotFound(projectId, clientName)); + } + } + })).toList(); + + threads.forEach(Thread::start); + for (var thread : threads) { + assertTrue(thread.join(Duration.ofSeconds(10))); + } + } + + public void testWaitForAsyncClientClose() throws Exception { + final CountDownLatch closeLatch = new CountDownLatch(1); + clientRefsCloseLatchRef.set(closeLatch); + + final List projectIds = randomList(1, 3, ESTestCase::randomUniqueProjectId); + final int iterations = between(3, 8); + + final List clientRefs = new ArrayList<>(); + for (int i = 0; i < iterations; i++) { + for (var projectId : projectIds) { + final List subsetOfClientNames = randomNonEmptySubsetOf(clientNames); + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, subsetOfClientNames.toArray(String[]::new))); + subsetOfClientNames.forEach(clientName -> { + final var newClient = s3PerProjectClientManager.client(projectId, clientName); + clientRefs.add(newClient); + newClient.decRef(); + }); + if (randomBoolean() && randomBoolean()) { + removeProjectFromClusterState(projectId); + } + } + } + + final Thread thread = new Thread(() -> s3Service.close()); + thread.start(); + + assertBusy(() -> assertTrue(closeInternalInvoked.get())); + Thread.sleep(between(0, 100)); + assertFalse(s3PerProjectClientManager.getClientsCloseListener().isDone()); + + closeLatch.countDown(); + assertTrue(thread.join(Duration.ofSeconds(10))); + assertTrue(s3PerProjectClientManager.getClientsCloseListener().isDone()); + clientRefs.forEach(clientRef -> assertFalse(clientRef.hasReferences())); + } + + public void testClusterAndProjectClients() { + final ProjectId projectId = randomUniqueProjectId(); + final String clientName = randomFrom(clientNames); + final boolean configureProjectClientsFirst = randomBoolean(); + if (configureProjectClientsFirst) { + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); + } + + final var repositoryMetadata = new RepositoryMetadata( + randomIdentifier(), + "s3", + Settings.builder().put("client", clientName).build() + ); + + final AmazonS3Reference clusterClient = s3Service.client(null, repositoryMetadata); + if (configureProjectClientsFirst == false) { + assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + } + clusterClient.decRef(); + + if (configureProjectClientsFirst == false) { + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); + } + final AmazonS3Reference projectClient = s3Service.client(projectId, repositoryMetadata); + assertThat(projectClient, not(sameInstance(clusterClient))); + projectClient.decRef(); + + s3Service.onBlobStoreClose(null); + assertFalse(clusterClient.hasReferences()); + assertTrue(projectClient.hasReferences()); + + s3Service.onBlobStoreClose(projectId); + assertFalse(projectClient.hasReferences()); + } + + public void testProjectClientsDisabled() { + final S3Service s3ServiceWithNoProjectSupport = new S3Service( + mock(Environment.class), + clusterService, + TestProjectResolvers.DEFAULT_PROJECT_ONLY, + mock(ResourceWatcherService.class), + () -> Region.of("es-test-region") + ) { + }; + s3ServiceWithNoProjectSupport.refreshAndClearCache(S3ClientSettings.load(clusterService.getSettings())); + s3ServiceWithNoProjectSupport.start(); + assertNull(s3ServiceWithNoProjectSupport.getS3PerProjectClientManager()); + + final String clientName = randomFrom(clientNames); + final var repositoryMetadata = new RepositoryMetadata( + randomIdentifier(), + "s3", + Settings.builder().put("client", clientName).build() + ); + final AmazonS3Reference clientRef = s3ServiceWithNoProjectSupport.client(ProjectId.DEFAULT, repositoryMetadata); + clientRef.decRef(); + s3ServiceWithNoProjectSupport.close(); + assertFalse(clientRef.hasReferences()); + } + + private void assertProjectClientSettings(ProjectId projectId, String... clientNames) { + final var clientsHolder = s3PerProjectClientManager.getProjectClientsHolders().get(projectId); + assertNotNull(clientsHolder); + final Map s3ClientSettingsMap = clientsHolder.clientSettings(); + assertThat(s3ClientSettingsMap.keySet(), containsInAnyOrder(clientNames)); + + for (var clientName : clientNames) { + final S3ClientSettings projectClientSettings = s3ClientSettingsMap.get(clientName); + final S3ClientSettings clusterClientSettings = clusterClientsSettings.get(clientName); + assertNotNull(clusterClientSettings); + + // Picks up the correct project scoped credentials + assertThat( + projectClientSettings.credentials, + equalTo( + AwsBasicCredentials.create(projectClientAccessKey(projectId, clientName), projectClientSecretKey(projectId, clientName)) + ) ); - assertThat(e.getMessage(), containsString("client [backup] does not exist")); + assertThat(projectClientSettings.credentials, not(equalTo(clusterClientSettings.credentials))); + // Inherit setting override from the cluster client of the same name + assertThat(projectClientSettings.maxRetries, equalTo(clusterClientSettings.maxRetries)); + assertThat(projectClientSettings.maxConnections, equalTo(clusterClientSettings.maxConnections)); + assertThat(projectClientSettings.readTimeoutMillis, equalTo(clusterClientSettings.readTimeoutMillis)); } + } - { - final ProjectId anotherProjectId = randomUniqueProjectId(); - final IllegalArgumentException e = expectThrows( - IllegalArgumentException.class, - () -> s3PerProjectClientManager.client(anotherProjectId, repositoryMetadata) + private void assertClientCredentials(ProjectId projectId, String clientName, AmazonS3Reference clientRef) { + try { + final AwsCredentialsIdentity awsCredentialsIdentity = clientRef.client() + .serviceClientConfiguration() + .credentialsProvider() + .resolveIdentity() + .get(); + assertThat(awsCredentialsIdentity.accessKeyId(), equalTo(projectClientAccessKey(projectId, clientName))); + assertThat(awsCredentialsIdentity.secretAccessKey(), equalTo(projectClientSecretKey(projectId, clientName))); + } catch (InterruptedException | ExecutionException e) { + fail(e, "unexpected exception"); + } + } + + private void assertClientNotFound(ProjectId projectId, String clientName) { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> s3PerProjectClientManager.client(projectId, clientName) + ); + assertThat( + e.getMessage(), + anyOf( + containsString("no s3 client is configured for project [" + projectId + "]"), + containsString("s3 client [" + clientName + "] does not exist for project [" + projectId + "]") + ) + ); + } + + private void updateProjectInClusterState(ProjectId projectId, Map projectClientSecrets) { + final var mockSecureSettings = new MockSecureSettings(); + projectClientSecrets.forEach((k, v) -> mockSecureSettings.setFile(k, v.getBytes(StandardCharsets.UTF_8))); + // Sometimes add an unrelated project secret + if (randomBoolean() && randomBoolean()) { + mockSecureSettings.setFile( + Strings.join(randomList(1, 5, ESTestCase::randomIdentifier), "."), + randomByteArrayOfLength(between(8, 20)) ); - assertThat(e.getMessage(), containsString("project [" + anotherProjectId + "] does not exist")); + } + final var secureClusterStateSettings = new SecureClusterStateSettings(mockSecureSettings); + + synchronized (this) { + final ClusterState initialState = clusterService.state(); + final ProjectMetadata.Builder projectBuilder = initialState.metadata().hasProject(projectId) + ? ProjectMetadata.builder(initialState.metadata().getProject(projectId)) + : ProjectMetadata.builder(projectId); + + if (secureClusterStateSettings.getSettingNames().isEmpty() == false + || projectBuilder.getCustom(ProjectSecrets.TYPE) != null + || randomBoolean()) { + projectBuilder.putCustom(ProjectSecrets.TYPE, new ProjectSecrets(secureClusterStateSettings)); + } + + final ClusterState stateWithProject = ClusterState.builder(initialState).putProjectMetadata(projectBuilder).build(); + ClusterServiceUtils.setState(clusterService, stateWithProject); } } - private static RepositoryMetadata createRepositoryMetadata(String clientName) { - return new RepositoryMetadata(randomIdentifier(), "s3", Settings.builder().put("client", clientName).build()); + private void removeProjectFromClusterState(ProjectId projectId) { + synchronized (this) { + final ClusterState initialState = clusterService.state(); + final ClusterState stateWithoutProject = ClusterState.builder(initialState) + .metadata(Metadata.builder(initialState.metadata()).removeProject(projectId)) + .routingTable(GlobalRoutingTable.builder(initialState.globalRoutingTable()).removeProject(projectId).build()) + .build(); + ClusterServiceUtils.setState(clusterService, stateWithoutProject); + } + } + + private Map newProjectClientsSecrets(ProjectId projectId, String... clientNames) { + s3SecretsIdGenerators.computeIfAbsent(projectId, ignored -> new AtomicInteger(0)).incrementAndGet(); + final Map m = new HashMap<>(); + Arrays.stream(clientNames).forEach(clientName -> { + m.put("s3.client." + clientName + ".access_key", projectClientAccessKey(projectId, clientName)); + m.put("s3.client." + clientName + ".secret_key", projectClientSecretKey(projectId, clientName)); + }); + return Map.copyOf(m); + } + + private String projectClientAccessKey(ProjectId projectId, String clientName) { + return projectId + "_" + clientName + "_access_key_" + s3SecretsIdGenerators.get(projectId).get(); + } + + private String projectClientSecretKey(ProjectId projectId, String clientName) { + return projectId + "_" + clientName + "_secret_key_" + s3SecretsIdGenerators.get(projectId).get(); } + private static class DummySdkHttpClient implements SdkHttpClient { + + static final SdkHttpClient INSTANCE = new DummySdkHttpClient(); + + @Override + public ExecutableHttpRequest prepareRequest(HttpExecuteRequest request) { + return null; + } + + @Override + public void close() {} + } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java b/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java index a821e276fb64e..338201861cf32 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java @@ -23,6 +23,7 @@ import java.util.EnumSet; import java.util.Iterator; import java.util.Objects; +import java.util.Set; /** * Secrets that are stored in project state as a {@link Metadata.ProjectCustom} @@ -55,6 +56,10 @@ public SecureSettings getSettings() { return new SecureClusterStateSettings(settings); } + public Set getSettingNames() { + return settings.getSettingNames(); + } + @Override public Iterator toXContentChunked(ToXContent.Params params) { // No need to persist in index or return to user, so do not serialize the secrets From a318fc89047af25c394df757c35c939de6b68272 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Sat, 3 May 2025 18:42:21 +1000 Subject: [PATCH 03/24] tweaks --- .../repositories/s3/S3PerProjectClientManager.java | 12 ++++++------ .../org/elasticsearch/repositories/s3/S3Service.java | 1 + .../common/settings/ProjectSecrets.java | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index b3e248e61fdcb..76c60611bd0f6 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -45,14 +45,14 @@ public class S3PerProjectClientManager implements ClusterStateListener { private final Settings nodeS3Settings; private final Function clientBuilder; private final Executor executor; - // A map of projectId to clients holder. Adding to and removing from the map happen only in the cluster state listener thread. + // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. private final Map projectClientsHolders; - // Listener for tracking ongoing async closing of obsolete clients. Updated only in the cluster state listener thread. + // Listener for tracking ongoing async closing of obsolete clients. Updated only in the applier thread. private volatile SubscribableListener clientsCloseListener = null; S3PerProjectClientManager(Settings settings, Function clientBuilder, Executor executor) { this.nodeS3Settings = Settings.builder() - .put(settings.getByPrefix(S3_SETTING_PREFIX), false) // not use any cluster scoped secrets + .put(settings.getByPrefix(S3_SETTING_PREFIX), false) // not rely on any cluster scoped secrets .normalizePrefix(S3_SETTING_PREFIX) .build(); this.clientBuilder = clientBuilder; @@ -72,7 +72,7 @@ public void clusterChanged(ClusterChangedEvent event) { final List clientsHoldersToClose = new ArrayList<>(); for (var project : currentProjects.values()) { final ProjectSecrets projectSecrets = project.custom(ProjectSecrets.TYPE); - // Project secrets can be null when node restarts. It may not have s3 credentials if s3 is not used. + // Project secrets can be null when node restarts. It may not have any s3 credentials if s3 is not in use. if (projectSecrets == null || projectSecrets.getSettingNames().stream().noneMatch(key -> key.startsWith("s3."))) { // Most likely there won't be any existing client, but attempt to remove it anyway just in case final ClientsHolder removed = projectClientsHolders.remove(project.id()); @@ -126,7 +126,7 @@ public void clusterChanged(ClusterChangedEvent event) { clientsHoldersToClose.add(removed); } } - // Close stale clients asynchronously without blocking the cluster state thread + // Close stale clients asynchronously without blocking the applier thread if (clientsHoldersToClose.isEmpty() == false) { final var currentClientsCloseListener = new SubscribableListener(); final var previousClientsCloseListener = clientsCloseListener; @@ -182,7 +182,7 @@ public void close() { currentClientsCloseListener.addListener(ActionListener.running(latch::countDown)); try { if (latch.await(1, TimeUnit.MINUTES) == false) { - logger.warn("async closing of s3 clients timed out"); + logger.warn("Waiting for async closing of s3 clients timed out"); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index a1f3cdc78d78e..1b7254359277e 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -227,6 +227,7 @@ public AmazonS3Reference client(@Nullable ProjectId projectId, RepositoryMetadat } } + // visible for tests protected AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); Releasable toRelease = httpClient::close; diff --git a/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java b/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java index 338201861cf32..2dc3a1c0ee961 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ProjectSecrets.java @@ -57,7 +57,7 @@ public SecureSettings getSettings() { } public Set getSettingNames() { - return settings.getSettingNames(); + return Collections.unmodifiableSet(settings.getSettingNames()); } @Override From c3b36e71d7b2170e15e76e551324b771500c48d3 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Mon, 5 May 2025 11:31:35 +1000 Subject: [PATCH 04/24] add jira issue --- .../repositories/s3/S3PerProjectClientManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index 76c60611bd0f6..ded70341e11ca 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -84,7 +84,7 @@ public void clusterChanged(ClusterChangedEvent event) { final Settings currentSettings = Settings.builder() // merge with static settings such as max retries etc - // TODO: We may need to update this if per-project settings decide to support hierarchical overrides + // TODO: https://elasticco.atlassian.net/browse/ES-11716 Consider change this to use per-project settings .put(nodeS3Settings) .setSecureSettings(projectSecrets.getSettings()) .build(); From fdc9c1d82ac3e50cfc211106fcfc8564c906c097 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Fri, 9 May 2025 13:15:49 +1000 Subject: [PATCH 05/24] applier --- .../repositories/s3/S3PerProjectClientManager.java | 7 ++++--- .../java/org/elasticsearch/repositories/s3/S3Service.java | 2 +- .../org/elasticsearch/repositories/s3/S3ServiceTests.java | 5 +++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index ded70341e11ca..a7fd8d9462357 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -12,7 +12,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; import org.elasticsearch.common.settings.ProjectSecrets; @@ -37,7 +37,7 @@ import java.util.function.Function; import java.util.stream.Collectors; -public class S3PerProjectClientManager implements ClusterStateListener { +public class S3PerProjectClientManager implements ClusterStateApplier { private static final Logger logger = LogManager.getLogger(S3PerProjectClientManager.class); private static final String S3_SETTING_PREFIX = "s3."; @@ -65,7 +65,8 @@ Map getProjectClientsHolders() { return Map.copyOf(projectClientsHolders); } - public void clusterChanged(ClusterChangedEvent event) { + @Override + public void applyClusterState(ClusterChangedEvent event) { final Map currentProjects = event.state().metadata().projects(); final var updatedPerProjectClients = new HashMap(); diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 1b7254359277e..274d7feed1761 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -155,7 +155,7 @@ class S3Service extends AbstractLifecycleComponent { this::buildClientReference, clusterService.threadPool().generic() ); - clusterService.addListener(s3PerProjectClientManager); + clusterService.addHighPriorityApplier(s3PerProjectClientManager); } else { s3PerProjectClientManager = null; } diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java index 778a55098bf20..4d119f2353092 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java @@ -319,10 +319,11 @@ public void testEndpointOverrideSchemeUsesHttpIfHttpProtocolSpecified() { See the breaking changes documentation for the next major version.""", clientName)); } - private static URI getEndpointUri(Settings.Builder settings, String clientName) { + private URI getEndpointUri(Settings.Builder settings, String clientName) { return new S3Service( mock(Environment.class), - Settings.EMPTY, + ClusterServiceUtils.createClusterService(threadPool), + TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of(randomIdentifier()) ).buildClient(S3ClientSettings.getClientSettings(settings.build(), clientName), mock(SdkHttpClient.class)) From 0f1bdd5b247c7af161d907c33603fb5f2b4a9c39 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Tue, 20 May 2025 18:19:42 +1000 Subject: [PATCH 06/24] abstract runnable --- .../s3/S3PerProjectClientManager.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index a7fd8d9462357..affa519e8c2cc 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.settings.ProjectSecrets; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.Maps; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Nullable; import org.elasticsearch.logging.LogManager; @@ -143,9 +144,18 @@ public void applyClusterState(ClusterChangedEvent event) { } private void closeClientsAsync(List clientsHoldersToClose, ActionListener listener) { - executor.execute(() -> { - IOUtils.closeWhileHandlingException(clientsHoldersToClose); - listener.onResponse(null); + executor.execute(new AbstractRunnable() { + @Override + protected void doRun() throws Exception { + IOUtils.closeWhileHandlingException(clientsHoldersToClose); + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + logger.warn("Failed to close s3 clients", e); + listener.onFailure(e); + } }); } From 9bc5d0939df608d9c3b372024535ca3546b59b33 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Tue, 20 May 2025 18:50:19 +1000 Subject: [PATCH 07/24] nodeSettings --- .../org/elasticsearch/repositories/s3/S3Service.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 548e32fd59f29..dcea69b4f8485 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -137,7 +137,7 @@ class S3Service extends AbstractLifecycleComponent { ResourceWatcherService resourceWatcherService, Supplier defaultRegionSupplier ) { - final Settings settings = clusterService.getSettings(); + final Settings nodeSettings = clusterService.getSettings(); webIdentityTokenCredentialsProvider = new CustomWebIdentityTokenCredentialsProvider( environment, System::getenv, @@ -145,13 +145,13 @@ class S3Service extends AbstractLifecycleComponent { Clock.systemUTC(), resourceWatcherService ); - compareAndExchangeTimeToLive = REPOSITORY_S3_CAS_TTL_SETTING.get(settings); - compareAndExchangeAntiContentionDelay = REPOSITORY_S3_CAS_ANTI_CONTENTION_DELAY_SETTING.get(settings); - isStateless = DiscoveryNode.isStateless(settings); + compareAndExchangeTimeToLive = REPOSITORY_S3_CAS_TTL_SETTING.get(nodeSettings); + compareAndExchangeAntiContentionDelay = REPOSITORY_S3_CAS_ANTI_CONTENTION_DELAY_SETTING.get(nodeSettings); + isStateless = DiscoveryNode.isStateless(nodeSettings); defaultRegionSetter = new RunOnce(() -> defaultRegion = defaultRegionSupplier.get()); if (projectResolver.supportsMultipleProjects()) { s3PerProjectClientManager = new S3PerProjectClientManager( - settings, + nodeSettings, this::buildClientReference, clusterService.threadPool().generic() ); From 55c697cccd8ffbc23db42c52111079dd5a786c03 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 12:03:01 +1000 Subject: [PATCH 08/24] generic --- .../s3/S3PerProjectClientManager.java | 125 ++++++++++++------ .../s3/S3PerProjectClientManagerTests.java | 2 +- 2 files changed, 88 insertions(+), 39 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java index affa519e8c2cc..6ed0d0b23777d 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java @@ -15,6 +15,7 @@ import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.common.settings.ProjectSecrets; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.Maps; @@ -47,7 +48,7 @@ public class S3PerProjectClientManager implements ClusterStateApplier { private final Function clientBuilder; private final Executor executor; // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. - private final Map projectClientsHolders; + private final Map> projectClientsHolders; // Listener for tracking ongoing async closing of obsolete clients. Updated only in the applier thread. private volatile SubscribableListener clientsCloseListener = null; @@ -62,7 +63,7 @@ public class S3PerProjectClientManager implements ClusterStateApplier { } // visible for tests - Map getProjectClientsHolders() { + Map> getProjectClientsHolders() { return Map.copyOf(projectClientsHolders); } @@ -70,16 +71,17 @@ Map getProjectClientsHolders() { public void applyClusterState(ClusterChangedEvent event) { final Map currentProjects = event.state().metadata().projects(); - final var updatedPerProjectClients = new HashMap(); - final List clientsHoldersToClose = new ArrayList<>(); + final var updatedPerProjectClients = new HashMap(); + final List clientsHoldersToClose = new ArrayList<>(); for (var project : currentProjects.values()) { final ProjectSecrets projectSecrets = project.custom(ProjectSecrets.TYPE); // Project secrets can be null when node restarts. It may not have any s3 credentials if s3 is not in use. if (projectSecrets == null || projectSecrets.getSettingNames().stream().noneMatch(key -> key.startsWith("s3."))) { // Most likely there won't be any existing client, but attempt to remove it anyway just in case - final ClientsHolder removed = projectClientsHolders.remove(project.id()); + final var removed = projectClientsHolders.remove(project.id()); if (removed != null) { - clientsHoldersToClose.add(removed); + assert removed instanceof PerProjectClientsHolder; + clientsHoldersToClose.add((PerProjectClientsHolder) removed); } continue; } @@ -109,7 +111,7 @@ public void applyClusterState(ClusterChangedEvent event) { // TODO: If performance is an issue, we may consider comparing just the relevant project secrets for new or updated clients // and avoid building the clientSettings if (newOrUpdated(project.id(), clientSettings)) { - updatedPerProjectClients.put(project.id(), new ClientsHolder(project.id(), clientSettings)); + updatedPerProjectClients.put(project.id(), new PerProjectClientsHolder(project.id(), clientSettings)); } } @@ -117,15 +119,16 @@ public void applyClusterState(ClusterChangedEvent event) { for (var projectId : updatedPerProjectClients.keySet()) { final var old = projectClientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); if (old != null) { - clientsHoldersToClose.add(old); + assert old instanceof PerProjectClientsHolder; + clientsHoldersToClose.add((PerProjectClientsHolder) old); } } // removed projects for (var projectId : projectClientsHolders.keySet()) { if (currentProjects.containsKey(projectId) == false) { final var removed = projectClientsHolders.remove(projectId); - assert removed != null; - clientsHoldersToClose.add(removed); + assert removed instanceof PerProjectClientsHolder; + clientsHoldersToClose.add((PerProjectClientsHolder) removed); } } // Close stale clients asynchronously without blocking the applier thread @@ -143,7 +146,7 @@ public void applyClusterState(ClusterChangedEvent event) { } } - private void closeClientsAsync(List clientsHoldersToClose, ActionListener listener) { + private void closeClientsAsync(List clientsHoldersToClose, ActionListener listener) { executor.execute(new AbstractRunnable() { @Override protected void doRun() throws Exception { @@ -165,7 +168,9 @@ public AmazonS3Reference client(ProjectId projectId, String clientName) { if (clientsHolder == null) { throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); } - return clientsHolder.client(clientName); + return clientsHolder.client( + new RepositoryMetadata("repo", S3Repository.TYPE, Settings.builder().put(S3Repository.CLIENT_NAME.getKey(), clientName).build()) + ); } /** @@ -212,46 +217,44 @@ private boolean newOrUpdated(ProjectId projectId, Map if (old == null) { return true; } - return currentClientSettings.equals(old.clientSettings()) == false; + return currentClientSettings.equals(old.allClientSettings()) == false; } - /** - * Holder class of s3 clients for a single project. It is instantiated in the cluster state thread with client - * settings. The clients are created and cached lazily when the {@link #client(String)} method is called. - * Cached clients are closed and cleared out when the {@link #clearCache()} method is called. Subsequent calls to - * {@link #client(String)} will recreate them. The call to {@link #close()} method clears the cache as well but - * also flags the holder to be closed so that no new clients can be created. - */ - final class ClientsHolder implements Closeable { - private final AtomicBoolean closed = new AtomicBoolean(false); - private final ProjectId projectId; - private final Map clientSettings; - // Client name -> client reference - private volatile Map clientsCache = Collections.emptyMap(); + abstract class ClientsHolder implements Closeable { + protected final ProjectId projectId; + protected final AtomicBoolean closed = new AtomicBoolean(false); + protected volatile Map clientsCache = Collections.emptyMap(); - ClientsHolder(ProjectId projectId, Map clientSettings) { + ClientsHolder(ProjectId projectId) { this.projectId = projectId; - this.clientSettings = clientSettings; } - Map clientSettings() { - return clientSettings; - } + abstract K clientKey(RepositoryMetadata repositoryMetadata); - AmazonS3Reference client(String clientName) { - final var clientReference = clientsCache.get(clientName); + abstract String clientName(K clientKey); + + abstract S3ClientSettings singleClientSettings(K clientKey); + + abstract Map allClientSettings(); + + AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { + final var clientKey = clientKey(repositoryMetadata); + + final var clientReference = clientsCache.get(clientKey); // It is ok to retrieve an existing client when the cache is being cleared or the holder is closing. // As long as there are paired incRef/decRef calls, the client will be closed when the last reference is released // by either the caller of this method or the clearCache() method. if (clientReference != null && clientReference.tryIncRef()) { return clientReference; } - final var settings = clientSettings.get(clientName); + final var settings = singleClientSettings(clientKey); if (settings == null) { - throw new IllegalArgumentException("s3 client [" + clientName + "] does not exist for project [" + projectId + "]"); + throw new IllegalArgumentException( + "s3 client [" + clientName(clientKey) + "] does not exist for project [" + projectId + "]" + ); } synchronized (this) { - final var existing = clientsCache.get(clientName); + final var existing = clientsCache.get(clientKey); if (existing != null && existing.tryIncRef()) { return existing; } @@ -262,13 +265,13 @@ AmazonS3Reference client(String clientName) { // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. // The clearCache() will clear the newly added client. final var newClientReference = clientBuilder.apply(settings); - clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientName, newClientReference); + clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientKey, newClientReference); return newClientReference; } } /** - * Clear the cache by closing and clear out all clients. Subsequent {@link #client(String)} calls will recreate + * Clear the cache by closing and clear out all clients. Subsequent {@link #client(RepositoryMetadata)} calls will recreate * the clients and populate the cache again. */ synchronized void clearCache() { @@ -290,4 +293,50 @@ boolean isClosed() { return closed.get(); } } + + /** + * Holder class of s3 clients for a single project. It is instantiated in the cluster state thread with client + * settings. The clients are created and cached lazily when the {@link #client(String)} method is called. + * Cached clients are closed and cleared out when the {@link #clearCache()} method is called. Subsequent calls to + * {@link #client(String)} will recreate them. The call to {@link #close()} method clears the cache as well but + * also flags the holder to be closed so that no new clients can be created. + */ + final class PerProjectClientsHolder extends ClientsHolder { + private final Map clientSettings; + + PerProjectClientsHolder(ProjectId projectId, Map clientSettings) { + super(projectId); + this.clientSettings = clientSettings; + } + + @Override + Map allClientSettings() { + return clientSettings; + } + + @Override + String clientKey(RepositoryMetadata repositoryMetadata) { + return S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); + } + + @Override + String clientName(String clientKey) { + return clientKey; + } + + @Override + S3ClientSettings singleClientSettings(String clientKey) { + return clientSettings.get(clientKey); + } + + AmazonS3Reference client(String clientName) { + return client( + new RepositoryMetadata( + "repo", + S3Repository.TYPE, + Settings.builder().put(S3Repository.CLIENT_NAME.getKey(), clientName).build() + ) + ); + } + } } diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java index eff51187f784e..89294cfa4d800 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java @@ -374,7 +374,7 @@ public void testProjectClientsDisabled() { private void assertProjectClientSettings(ProjectId projectId, String... clientNames) { final var clientsHolder = s3PerProjectClientManager.getProjectClientsHolders().get(projectId); assertNotNull(clientsHolder); - final Map s3ClientSettingsMap = clientsHolder.clientSettings(); + final Map s3ClientSettingsMap = clientsHolder.allClientSettings(); assertThat(s3ClientSettingsMap.keySet(), containsInAnyOrder(clientNames)); for (var clientName : clientNames) { From 833e0859dcb01bb3db124f349cb619b1792370fc Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 14:42:09 +1000 Subject: [PATCH 09/24] consolidate --- ...ientManager.java => S3ClientsManager.java} | 277 ++++++++++++++---- .../repositories/s3/S3Service.java | 151 ++-------- ...rTests.java => S3ClientsManagerTests.java} | 74 +++-- .../repositories/s3/S3ServiceTests.java | 9 +- 4 files changed, 289 insertions(+), 222 deletions(-) rename modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/{S3PerProjectClientManager.java => S3ClientsManager.java} (55%) rename modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/{S3PerProjectClientManagerTests.java => S3ClientsManagerTests.java} (86%) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java similarity index 55% rename from modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java rename to modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index 6ed0d0b23777d..ae5b34749544f 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3PerProjectClientManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.cluster.metadata.ProjectMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ProjectSecrets; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.Maps; @@ -31,40 +32,43 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Objects; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; -public class S3PerProjectClientManager implements ClusterStateApplier { +import static java.util.Collections.emptyMap; - private static final Logger logger = LogManager.getLogger(S3PerProjectClientManager.class); +public class S3ClientsManager implements ClusterStateApplier { + + private static final Logger logger = LogManager.getLogger(S3ClientsManager.class); private static final String S3_SETTING_PREFIX = "s3."; private final Settings nodeS3Settings; private final Function clientBuilder; private final Executor executor; // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. - private final Map> projectClientsHolders; + private final Map> clientsHolders; // Listener for tracking ongoing async closing of obsolete clients. Updated only in the applier thread. private volatile SubscribableListener clientsCloseListener = null; - S3PerProjectClientManager(Settings settings, Function clientBuilder, Executor executor) { + S3ClientsManager( + Settings nodeSettings, + Function clientBuilder, + UnaryOperator>> clientsHoldersWrapper, + Executor executor + ) { this.nodeS3Settings = Settings.builder() - .put(settings.getByPrefix(S3_SETTING_PREFIX), false) // not rely on any cluster scoped secrets + .put(nodeSettings.getByPrefix(S3_SETTING_PREFIX), false) // not rely on any cluster scoped secrets .normalizePrefix(S3_SETTING_PREFIX) .build(); this.clientBuilder = clientBuilder; this.executor = executor; - this.projectClientsHolders = new ConcurrentHashMap<>(); - } - - // visible for tests - Map> getProjectClientsHolders() { - return Map.copyOf(projectClientsHolders); + this.clientsHolders = clientsHoldersWrapper.apply(Map.of(ProjectId.DEFAULT, new ClusterClientsHolder())); } @Override @@ -74,11 +78,15 @@ public void applyClusterState(ClusterChangedEvent event) { final var updatedPerProjectClients = new HashMap(); final List clientsHoldersToClose = new ArrayList<>(); for (var project : currentProjects.values()) { + // Skip the default project, it is handled differently with the ReloadablePlugin interface + if (ProjectId.DEFAULT.equals(project.id())) { + continue; + } final ProjectSecrets projectSecrets = project.custom(ProjectSecrets.TYPE); // Project secrets can be null when node restarts. It may not have any s3 credentials if s3 is not in use. if (projectSecrets == null || projectSecrets.getSettingNames().stream().noneMatch(key -> key.startsWith("s3."))) { // Most likely there won't be any existing client, but attempt to remove it anyway just in case - final var removed = projectClientsHolders.remove(project.id()); + final var removed = clientsHolders.remove(project.id()); if (removed != null) { assert removed instanceof PerProjectClientsHolder; clientsHoldersToClose.add((PerProjectClientsHolder) removed); @@ -117,16 +125,17 @@ public void applyClusterState(ClusterChangedEvent event) { // Updated projects for (var projectId : updatedPerProjectClients.keySet()) { - final var old = projectClientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); + final var old = clientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); if (old != null) { assert old instanceof PerProjectClientsHolder; clientsHoldersToClose.add((PerProjectClientsHolder) old); } } // removed projects - for (var projectId : projectClientsHolders.keySet()) { + for (var projectId : clientsHolders.keySet()) { if (currentProjects.containsKey(projectId) == false) { - final var removed = projectClientsHolders.remove(projectId); + assert ProjectId.DEFAULT.equals(projectId) == false; + final var removed = clientsHolders.remove(projectId); assert removed instanceof PerProjectClientsHolder; clientsHoldersToClose.add((PerProjectClientsHolder) removed); } @@ -162,26 +171,50 @@ public void onFailure(Exception e) { }); } - public AmazonS3Reference client(ProjectId projectId, String clientName) { - assert projectId != null && ProjectId.DEFAULT.equals(projectId) == false : projectId; - final var clientsHolder = projectClientsHolders.get(projectId); + // visible for tests + Map> getClientsHolders() { + return Map.copyOf(clientsHolders); + } + + void refreshAndClearCacheForClusterClients(Map clientsSettings) { + final var clientsHolder = clientsHolders.get(ProjectId.DEFAULT); + if (clientsHolder instanceof ClusterClientsHolder clusterClientsHolder) { + clusterClientsHolder.refreshAndClearCache(clientsSettings); + } else { + final String message = "expect cluster clients holder, got " + clientsHolder; + assert false : message; + throw new IllegalStateException(message); + } + } + + S3ClientSettings settingsForClient(ProjectId projectId, RepositoryMetadata repositoryMetadata) { + final var clientsHolder = clientsHolders.get(Objects.requireNonNull(projectId)); if (clientsHolder == null) { + assert ProjectId.DEFAULT.equals(projectId) == false; throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); } - return clientsHolder.client( - new RepositoryMetadata("repo", S3Repository.TYPE, Settings.builder().put(S3Repository.CLIENT_NAME.getKey(), clientName).build()) - ); + return clientsHolder.singleClientSettings(repositoryMetadata); + } + + AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetadata) { + final var clientsHolder = clientsHolders.get(Objects.requireNonNull(projectId)); + if (clientsHolder == null) { + assert ProjectId.DEFAULT.equals(projectId) == false; + throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); + } + return clientsHolder.client(repositoryMetadata); } /** * Similar to S3Service#releaseCachedClients but only clears the cache for the given project. * All clients for the project are closed and will be recreated on next access, also similar to S3Service#releaseCachedClients */ - public void releaseProjectClients(ProjectId projectId) { - assert projectId != null && ProjectId.DEFAULT.equals(projectId) == false : projectId; - final var old = projectClientsHolders.get(projectId); + void releaseCachedClients(ProjectId projectId) { + final var old = clientsHolders.get(Objects.requireNonNull(projectId)); if (old != null) { old.clearCache(); + } else { + assert ProjectId.DEFAULT.equals(projectId) == false; } } @@ -189,8 +222,8 @@ public void releaseProjectClients(ProjectId projectId) { * Shutdown the manager by closing all clients holders. This is called when the node is shutting down. * It attempts to wait (1 min) for any async client closing to complete. */ - public void close() { - IOUtils.closeWhileHandlingException(projectClientsHolders.values()); + void close() { + IOUtils.closeWhileHandlingException(clientsHolders.values()); final var currentClientsCloseListener = clientsCloseListener; if (currentClientsCloseListener != null && currentClientsCloseListener.isDone() == false) { // Wait for async clients closing to be completed @@ -213,33 +246,72 @@ SubscribableListener getClientsCloseListener() { } private boolean newOrUpdated(ProjectId projectId, Map currentClientSettings) { - final var old = projectClientsHolders.get(projectId); + final var old = clientsHolders.get(projectId); if (old == null) { return true; } return currentClientSettings.equals(old.allClientSettings()) == false; } + /** + * The base class of s3 clients holder for a project or the cluster. The clients are created and cached lazily when + * {@link #client(RepositoryMetadata)} method is called. Cached clients are closed and cleared out when + * {@link #clearCache()} method is called. Subsequent calls to {@link #client(RepositoryMetadata)} will + * recreate them. The call to {@link #close()} method clears the cache as well but also flags the holder to be + * closed so that no new clients can be created. + * Subclasses must define the type of the client key for the cache. + */ abstract class ClientsHolder implements Closeable { - protected final ProjectId projectId; - protected final AtomicBoolean closed = new AtomicBoolean(false); protected volatile Map clientsCache = Collections.emptyMap(); + private final AtomicBoolean closed = new AtomicBoolean(false); - ClientsHolder(ProjectId projectId) { - this.projectId = projectId; - } - + /** + * Determine the client key for retrieving the cached client. + * @param repositoryMetadata The repository metadata + * @return Key to a cached client. + */ abstract K clientKey(RepositoryMetadata repositoryMetadata); - abstract String clientName(K clientKey); - + /** + * Get the client settings for a given client key. + */ abstract S3ClientSettings singleClientSettings(K clientKey); + /** + * Get a map of client name to client settings for all clients. + */ abstract Map allClientSettings(); - AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { - final var clientKey = clientKey(repositoryMetadata); + /** + * Get the project id for which this clients holder is associated with. + */ + abstract ProjectId projectId(); + /** + * Determine the client name from the given repository metadata. + */ + String clientName(RepositoryMetadata repositoryMetadata) { + return S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); + } + + /** + * Similar to {@link #singleClientSettings(K)} but from the given repository metadata. + */ + S3ClientSettings singleClientSettings(RepositoryMetadata repositoryMetadata) { + return singleClientSettings(clientKey(repositoryMetadata)); + } + + /** + * Retrieves an {@link AmazonS3Reference} for the given repository metadata. If a cached client exists and can be + * referenced, it is returned. Otherwise, a new client is created, added to the cache, and returned. + * + * @param repositoryMetadata The metadata of the repository for which the Amazon S3 client is required. + * @return An {@link AmazonS3Reference} instance corresponding to the repository metadata. + * @throws IllegalArgumentException If no client settings exist for the given repository metadata. + * @throws IllegalStateException If the client manager is closed and a new client cannot be created. + */ + final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { + final var clientKey = clientKey(repositoryMetadata); final var clientReference = clientsCache.get(clientKey); // It is ok to retrieve an existing client when the cache is being cleared or the holder is closing. // As long as there are paired incRef/decRef calls, the client will be closed when the last reference is released @@ -247,12 +319,14 @@ AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { if (clientReference != null && clientReference.tryIncRef()) { return clientReference; } + final var settings = singleClientSettings(clientKey); if (settings == null) { throw new IllegalArgumentException( - "s3 client [" + clientName(clientKey) + "] does not exist for project [" + projectId + "]" + "s3 client [" + clientName(repositoryMetadata) + "] does not exist for project [" + projectId() + "]" ); } + synchronized (this) { final var existing = clientsCache.get(clientKey); if (existing != null && existing.tryIncRef()) { @@ -274,38 +348,40 @@ AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { * Clear the cache by closing and clear out all clients. Subsequent {@link #client(RepositoryMetadata)} calls will recreate * the clients and populate the cache again. */ - synchronized void clearCache() { + final synchronized void clearCache() { + // the clients will shutdown when they will not be used anymore IOUtils.closeWhileHandlingException(clientsCache.values()); + // clear previously cached clients, they will be built lazily clientsCache = Collections.emptyMap(); + doClearCache(); } + void doClearCache() {} + /** * Similar to {@link #clearCache()} but also flag the holder to be closed so that no new client can be created. */ - public void close() { + public final void close() { if (closed.compareAndSet(false, true)) { clearCache(); } } // visible for tests - boolean isClosed() { + final boolean isClosed() { return closed.get(); } } /** - * Holder class of s3 clients for a single project. It is instantiated in the cluster state thread with client - * settings. The clients are created and cached lazily when the {@link #client(String)} method is called. - * Cached clients are closed and cleared out when the {@link #clearCache()} method is called. Subsequent calls to - * {@link #client(String)} will recreate them. The call to {@link #close()} method clears the cache as well but - * also flags the holder to be closed so that no new clients can be created. + * S3 clients holder for a single project. The client cache is keyed by the client name. */ final class PerProjectClientsHolder extends ClientsHolder { + private final ProjectId projectId; private final Map clientSettings; PerProjectClientsHolder(ProjectId projectId, Map clientSettings) { - super(projectId); + this.projectId = projectId; this.clientSettings = clientSettings; } @@ -320,23 +396,106 @@ String clientKey(RepositoryMetadata repositoryMetadata) { } @Override - String clientName(String clientKey) { - return clientKey; + S3ClientSettings singleClientSettings(String clientKey) { + return clientSettings.get(clientKey); } @Override - S3ClientSettings singleClientSettings(String clientKey) { - return clientSettings.get(clientKey); + ProjectId projectId() { + return projectId; } + } + + /** + * S3 clients holder for the cluster. The client cache is keyed by the derived client settings. + * The derived client settings are computed by combining the static client settings with overrides from the repository metadata. + */ + final class ClusterClientsHolder extends ClientsHolder { - AmazonS3Reference client(String clientName) { - return client( - new RepositoryMetadata( - "repo", - S3Repository.TYPE, - Settings.builder().put(S3Repository.CLIENT_NAME.getKey(), clientName).build() - ) + /** + * Client settings calculated from static configuration and settings in the keystore. + */ + private volatile Map staticClientSettings = Map.of( + "default", + S3ClientSettings.getClientSettings(Settings.EMPTY, "default") + ); + + /** + * Client settings derived from those in {@link #staticClientSettings} by combining them with settings + * in the {@link RepositoryMetadata}. + */ + private volatile Map derivedClientSettings = emptyMap(); + + /** + * Either fetches {@link S3ClientSettings} for a given {@link RepositoryMetadata} from cached settings or creates them + * by overriding static client settings from {@link #staticClientSettings} with settings found in the repository metadata. + * @param repositoryMetadata Repository Metadata + * @return S3ClientSettings + */ + @Override + S3ClientSettings clientKey(RepositoryMetadata repositoryMetadata) { + final Settings settings = repositoryMetadata.settings(); + { + final S3ClientSettings existing = derivedClientSettings.get(settings); + if (existing != null) { + return existing; + } + } + final String clientName = S3Repository.CLIENT_NAME.get(settings); + final S3ClientSettings staticSettings = staticClientSettings.get(clientName); + if (staticSettings != null) { + synchronized (this) { + final S3ClientSettings existing = derivedClientSettings.get(settings); + if (existing != null) { + return existing; + } + final S3ClientSettings newSettings = staticSettings.refine(settings); + derivedClientSettings = Maps.copyMapWithAddedOrReplacedEntry(derivedClientSettings, settings, newSettings); + return newSettings; + } + } + throw new IllegalArgumentException( + "Unknown s3 client name [" + + clientName + + "]. Existing client configs: " + + Strings.collectionToDelimitedString(staticClientSettings.keySet(), ",") ); } + + @Override + S3ClientSettings singleClientSettings(S3ClientSettings clientKey) { + return clientKey; + } + + @Override + Map allClientSettings() { + return staticClientSettings; + } + + @Override + void doClearCache() { + // clear the derived settings, they will be built lazily + derivedClientSettings = emptyMap(); + } + + @Override + ProjectId projectId() { + return ProjectId.DEFAULT; + } + + /** + * Refreshes the settings for the AmazonS3 clients and clears the cache of + * existing clients. New clients will be built using these new settings. Old + * clients are usable until released. On release, they will be destroyed instead + * of being returned to the cache. + */ + synchronized void refreshAndClearCache(Map clientsSettings) { + // shutdown all unused clients + // others will shutdown on their respective release + clearCache(); // clears client cache and derived settings + this.staticClientSettings = Maps.ofEntries(clientsSettings.entrySet()); + assert this.staticClientSettings.containsKey("default") : "always at least have 'default'"; + /* clients are built lazily by {@link #client} */ + } } } diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index dcea69b4f8485..2e36c0ab2cf78 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -37,7 +37,6 @@ import org.apache.http.conn.DnsResolver; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.coordination.stateless.StoreHeartbeatService; import org.elasticsearch.cluster.metadata.ProjectId; @@ -49,8 +48,8 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.RunOnce; +import org.elasticsearch.core.FixForMultiProject; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -72,10 +71,11 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.function.UnaryOperator; -import static java.util.Collections.emptyMap; import static software.amazon.awssdk.core.SdkSystemSetting.AWS_ROLE_ARN; import static software.amazon.awssdk.core.SdkSystemSetting.AWS_ROLE_SESSION_NAME; import static software.amazon.awssdk.core.SdkSystemSetting.AWS_WEB_IDENTITY_TOKEN_FILE; @@ -96,21 +96,6 @@ class S3Service extends AbstractLifecycleComponent { TimeValue.timeValueHours(24), Setting.Property.NodeScope ); - private volatile Map clientsCache = emptyMap(); - - /** - * Client settings calculated from static configuration and settings in the keystore. - */ - private volatile Map staticClientSettings = Map.of( - "default", - S3ClientSettings.getClientSettings(Settings.EMPTY, "default") - ); - - /** - * Client settings derived from those in {@link #staticClientSettings} by combining them with settings - * in the {@link RepositoryMetadata}. - */ - private volatile Map derivedClientSettings = emptyMap(); private final Runnable defaultRegionSetter; private volatile Region defaultRegion; @@ -127,8 +112,7 @@ class S3Service extends AbstractLifecycleComponent { final TimeValue compareAndExchangeTimeToLive; final TimeValue compareAndExchangeAntiContentionDelay; final boolean isStateless; - @Nullable // if multi-project is disabled - private final S3PerProjectClientManager s3PerProjectClientManager; + private final S3ClientsManager s3ClientsManager; S3Service( Environment environment, @@ -149,22 +133,20 @@ class S3Service extends AbstractLifecycleComponent { compareAndExchangeAntiContentionDelay = REPOSITORY_S3_CAS_ANTI_CONTENTION_DELAY_SETTING.get(nodeSettings); isStateless = DiscoveryNode.isStateless(nodeSettings); defaultRegionSetter = new RunOnce(() -> defaultRegion = defaultRegionSupplier.get()); + s3ClientsManager = new S3ClientsManager( + nodeSettings, + this::buildClientReference, + projectResolver.supportsMultipleProjects() ? ConcurrentHashMap::new : UnaryOperator.identity(), + clusterService.threadPool().generic() + ); if (projectResolver.supportsMultipleProjects()) { - s3PerProjectClientManager = new S3PerProjectClientManager( - nodeSettings, - this::buildClientReference, - clusterService.threadPool().generic() - ); - clusterService.addHighPriorityApplier(s3PerProjectClientManager); - } else { - s3PerProjectClientManager = null; + clusterService.addHighPriorityApplier(s3ClientsManager); } } // visible to tests - @Nullable - S3PerProjectClientManager getS3PerProjectClientManager() { - return s3PerProjectClientManager; + S3ClientsManager getS3PerProjectClientManager() { + return s3ClientsManager; } /** @@ -174,43 +156,16 @@ S3PerProjectClientManager getS3PerProjectClientManager() { * of being returned to the cache. */ public synchronized void refreshAndClearCache(Map clientsSettings) { - // shutdown all unused clients - // others will shutdown on their respective release - releaseCachedClients(); - this.staticClientSettings = Maps.ofEntries(clientsSettings.entrySet()); - derivedClientSettings = emptyMap(); - assert this.staticClientSettings.containsKey("default") : "always at least have 'default'"; - /* clients are built lazily by {@link #client} */ + s3ClientsManager.refreshAndClearCacheForClusterClients(clientsSettings); } /** * Attempts to retrieve a client by its repository metadata and settings from the cache. * If the client does not exist it will be created. */ + @FixForMultiProject(description = "can be removed once blobstore is project aware") public AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { - final S3ClientSettings clientSettings = settings(repositoryMetadata); - { - final AmazonS3Reference clientReference = clientsCache.get(clientSettings); - if (clientReference != null && clientReference.tryIncRef()) { - return clientReference; - } - } - synchronized (this) { - final AmazonS3Reference existing = clientsCache.get(clientSettings); - if (existing != null && existing.tryIncRef()) { - return existing; - } - - if (lifecycle.started() == false) { - // doClose() calls releaseCachedClients() which is also synchronized (this) so if we're STARTED here then the client we - // create will definitely not leak on close. - throw new AlreadyClosedException("S3Service is in state [" + lifecycle + "]"); - } - - final AmazonS3Reference clientReference = buildClientReference(clientSettings); - clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientSettings, clientReference); - return clientReference; - } + return client(ProjectId.DEFAULT, repositoryMetadata); } /** @@ -221,17 +176,7 @@ public AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { * 2. Blobstore is cluster level (projectId = null) */ public AmazonS3Reference client(@Nullable ProjectId projectId, RepositoryMetadata repositoryMetadata) { - if (s3PerProjectClientManager == null) { - // Multi-Project is disabled and we have a single default project - assert ProjectId.DEFAULT.equals(projectId) : projectId; - return client(repositoryMetadata); - } else if (projectId == null) { - // Multi-Project is enabled and we are retrieving a client for the cluster level blobstore - return client(repositoryMetadata); - } else { - final String clientName = S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); - return s3PerProjectClientManager.client(projectId, clientName); - } + return s3ClientsManager.client(projectId == null ? ProjectId.DEFAULT : projectId, repositoryMetadata); } // visible for tests @@ -248,39 +193,8 @@ protected AmazonS3Reference buildClientReference(final S3ClientSettings clientSe } } - /** - * Either fetches {@link S3ClientSettings} for a given {@link RepositoryMetadata} from cached settings or creates them - * by overriding static client settings from {@link #staticClientSettings} with settings found in the repository metadata. - * @param repositoryMetadata Repository Metadata - * @return S3ClientSettings - */ S3ClientSettings settings(RepositoryMetadata repositoryMetadata) { - final Settings settings = repositoryMetadata.settings(); - { - final S3ClientSettings existing = derivedClientSettings.get(settings); - if (existing != null) { - return existing; - } - } - final String clientName = S3Repository.CLIENT_NAME.get(settings); - final S3ClientSettings staticSettings = staticClientSettings.get(clientName); - if (staticSettings != null) { - synchronized (this) { - final S3ClientSettings existing = derivedClientSettings.get(settings); - if (existing != null) { - return existing; - } - final S3ClientSettings newSettings = staticSettings.refine(settings); - derivedClientSettings = Maps.copyMapWithAddedOrReplacedEntry(derivedClientSettings, settings, newSettings); - return newSettings; - } - } - throw new IllegalArgumentException( - "Unknown s3 client name [" - + clientName - + "]. Existing client configs: " - + Strings.collectionToDelimitedString(staticClientSettings.keySet(), ",") - ); + return s3ClientsManager.settingsForClient(ProjectId.DEFAULT, repositoryMetadata); } // proxy for testing @@ -498,18 +412,9 @@ static AwsCredentialsProvider buildCredentials( } } - private synchronized void releaseCachedClients() { - // the clients will shutdown when they will not be used anymore - for (final AmazonS3Reference clientReference : clientsCache.values()) { - clientReference.decRef(); - } - // clear previously cached clients, they will be build lazily - clientsCache = emptyMap(); - derivedClientSettings = emptyMap(); - } - + @FixForMultiProject(description = "can be removed once blobstore is project aware") public void onBlobStoreClose() { - releaseCachedClients(); + onBlobStoreClose(ProjectId.DEFAULT); } /** @@ -519,16 +424,7 @@ public void onBlobStoreClose() { * 2. Blobstore is cluster level (projectId = null) */ public void onBlobStoreClose(@Nullable ProjectId projectId) { - if (s3PerProjectClientManager == null) { - // Multi-Project is disabled and we have a single default project - assert ProjectId.DEFAULT.equals(projectId) : projectId; - onBlobStoreClose(); - } else if (projectId == null) { - // Multi-Project is enabled and this is for the cluster level blobstore - onBlobStoreClose(); - } else { - s3PerProjectClientManager.releaseProjectClients(projectId); - } + s3ClientsManager.releaseCachedClients(projectId == null ? ProjectId.DEFAULT : projectId); } @Override @@ -541,10 +437,7 @@ protected void doStop() {} @Override public void doClose() throws IOException { - releaseCachedClients(); - if (s3PerProjectClientManager != null) { - s3PerProjectClientManager.close(); - } + s3ClientsManager.close(); webIdentityTokenCredentialsProvider.close(); } diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java similarity index 86% rename from modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java rename to modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index 89294cfa4d800..4807f7b23f455 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3PerProjectClientManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -59,9 +59,13 @@ import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; -public class S3PerProjectClientManagerTests extends ESTestCase { +public class S3ClientsManagerTests extends ESTestCase { private Map s3SecretsIdGenerators; private List clientNames; @@ -69,7 +73,7 @@ public class S3PerProjectClientManagerTests extends ESTestCase { private TestThreadPool threadPool; private ClusterService clusterService; private S3Service s3Service; - private S3PerProjectClientManager s3PerProjectClientManager; + private S3ClientsManager s3ClientsManager; private final AtomicReference clientRefsCloseLatchRef = new AtomicReference<>(); private final AtomicBoolean closeInternalInvoked = new AtomicBoolean(false); @@ -130,8 +134,8 @@ protected void closeInternal() { } }; s3Service.refreshAndClearCache(S3ClientSettings.load(settings)); - s3PerProjectClientManager = s3Service.getS3PerProjectClientManager(); - assertNotNull(s3PerProjectClientManager); + s3ClientsManager = s3Service.getS3PerProjectClientManager(); + assertNotNull(s3ClientsManager); s3Service.start(); } @@ -141,13 +145,13 @@ public void tearDown() throws Exception { s3Service.close(); clusterService.close(); threadPool.close(); - final var clientsCloseListener = s3PerProjectClientManager.getClientsCloseListener(); + final var clientsCloseListener = s3ClientsManager.getClientsCloseListener(); assertTrue(clientsCloseListener == null || clientsCloseListener.isDone()); - s3PerProjectClientManager.getProjectClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); + s3ClientsManager.getClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); } public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { - assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); final ProjectId projectId = randomUniqueProjectId(); // No project secrets at all @@ -155,7 +159,7 @@ public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { clusterService, ClusterState.builder(clusterService.state()).putProjectMetadata(ProjectMetadata.builder(projectId)).build() ); - assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); // Project secrets but no s3 credentials final var mockSecureSettings = new MockSecureSettings(); @@ -172,7 +176,7 @@ public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { ) .build() ); - assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); } public void testClientsLifeCycleForSingleProject() { @@ -187,17 +191,17 @@ public void testClientsLifeCycleForSingleProject() { assertProjectClientSettings(projectId, clientName); // Retrieve client for the 1st time - final AmazonS3Reference initialClient = s3PerProjectClientManager.client(projectId, clientName); + final AmazonS3Reference initialClient = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); assertClientCredentials(projectId, clientName, initialClient); // Client is cached when retrieved again - assertThat(initialClient, sameInstance(s3PerProjectClientManager.client(projectId, clientName))); + assertThat(initialClient, sameInstance(s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)))); // Client not configured cannot be accessed assertClientNotFound(projectId, anotherClientName); // Client should be released and recreated again on access - s3PerProjectClientManager.releaseProjectClients(projectId); - final AmazonS3Reference clientUpdated = s3PerProjectClientManager.client(projectId, clientName); + s3ClientsManager.releaseCachedClients(projectId); + final AmazonS3Reference clientUpdated = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); assertThat(clientUpdated, not(sameInstance(initialClient))); clientUpdated.decRef(); @@ -209,12 +213,12 @@ public void testClientsLifeCycleForSingleProject() { // Update client secrets should release and recreate the client updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName, anotherClientName)); assertProjectClientSettings(projectId, clientName, anotherClientName); - final AmazonS3Reference clientUpdateAgain = s3PerProjectClientManager.client(projectId, clientName); + final AmazonS3Reference clientUpdateAgain = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); assertThat(clientUpdateAgain, not(sameInstance(clientUpdated))); clientUpdateAgain.decRef(); // A different client for a different client name - final AmazonS3Reference antherClient = s3PerProjectClientManager.client(projectId, anotherClientName); + final AmazonS3Reference antherClient = s3ClientsManager.client(projectId, createRepositoryMetadata(anotherClientName)); assertClientCredentials(projectId, anotherClientName, antherClient); assertThat(antherClient, not(sameInstance(clientUpdateAgain))); antherClient.decRef(); @@ -240,18 +244,18 @@ public void testClientsForMultipleProjects() throws InterruptedException { assertProjectClientSettings(projectId, clientNames.toArray(String[]::new)); for (var clientName : shuffledList(clientNames)) { - try (var clientRef = s3PerProjectClientManager.client(projectId, clientName)) { + try (var clientRef = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName))) { assertClientCredentials(projectId, clientName, clientRef); } } if (randomBoolean()) { final Map previousClientRefs = clientNames.stream() - .map(clientName -> Map.entry(clientName, s3PerProjectClientManager.client(projectId, clientName))) + .map(clientName -> Map.entry(clientName, s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - s3PerProjectClientManager.releaseProjectClients(projectId); + s3ClientsManager.releaseCachedClients(projectId); previousClientRefs.forEach((clientName, previousClientRef) -> { - final AmazonS3Reference currentClientRef = s3PerProjectClientManager.client(projectId, clientName); + final AmazonS3Reference currentClientRef = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); assertThat(currentClientRef, not(sameInstance(previousClientRef))); assertClientCredentials(projectId, clientName, currentClientRef); currentClientRef.decRef(); @@ -263,7 +267,7 @@ public void testClientsForMultipleProjects() throws InterruptedException { } else { removeProjectFromClusterState(projectId); } - assertThat(s3PerProjectClientManager.getProjectClientsHolders(), not(hasKey(projectId))); + assertThat(getClientsHoldersExcludeDefaultProject(), not(hasKey(projectId))); clientNames.forEach(clientName -> assertClientNotFound(projectId, clientName)); } } @@ -288,7 +292,7 @@ public void testWaitForAsyncClientClose() throws Exception { final List subsetOfClientNames = randomNonEmptySubsetOf(clientNames); updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, subsetOfClientNames.toArray(String[]::new))); subsetOfClientNames.forEach(clientName -> { - final var newClient = s3PerProjectClientManager.client(projectId, clientName); + final var newClient = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); clientRefs.add(newClient); newClient.decRef(); }); @@ -303,11 +307,11 @@ public void testWaitForAsyncClientClose() throws Exception { assertBusy(() -> assertTrue(closeInternalInvoked.get())); Thread.sleep(between(0, 100)); - assertFalse(s3PerProjectClientManager.getClientsCloseListener().isDone()); + assertFalse(s3ClientsManager.getClientsCloseListener().isDone()); closeLatch.countDown(); assertTrue(thread.join(Duration.ofSeconds(10))); - assertTrue(s3PerProjectClientManager.getClientsCloseListener().isDone()); + assertTrue(s3ClientsManager.getClientsCloseListener().isDone()); clientRefs.forEach(clientRef -> assertFalse(clientRef.hasReferences())); } @@ -327,7 +331,7 @@ public void testClusterAndProjectClients() { final AmazonS3Reference clusterClient = s3Service.client(null, repositoryMetadata); if (configureProjectClientsFirst == false) { - assertThat(s3PerProjectClientManager.getProjectClientsHolders(), anEmptyMap()); + assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); } clusterClient.decRef(); @@ -347,6 +351,7 @@ public void testClusterAndProjectClients() { } public void testProjectClientsDisabled() { + final var clusterService = spy(this.clusterService); final S3Service s3ServiceWithNoProjectSupport = new S3Service( mock(Environment.class), clusterService, @@ -357,8 +362,10 @@ public void testProjectClientsDisabled() { }; s3ServiceWithNoProjectSupport.refreshAndClearCache(S3ClientSettings.load(clusterService.getSettings())); s3ServiceWithNoProjectSupport.start(); - assertNull(s3ServiceWithNoProjectSupport.getS3PerProjectClientManager()); + assertNotNull(s3ServiceWithNoProjectSupport.getS3PerProjectClientManager()); + verify(clusterService, never()).addHighPriorityApplier(any()); + // Cluster client still works final String clientName = randomFrom(clientNames); final var repositoryMetadata = new RepositoryMetadata( randomIdentifier(), @@ -371,8 +378,17 @@ public void testProjectClientsDisabled() { assertFalse(clientRef.hasReferences()); } + private Map> getClientsHoldersExcludeDefaultProject() { + final var holders = s3ClientsManager.getClientsHolders(); + assertThat(holders, hasKey(ProjectId.DEFAULT)); + return holders.entrySet() + .stream() + .filter(entry -> entry.getKey() != ProjectId.DEFAULT) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + } + private void assertProjectClientSettings(ProjectId projectId, String... clientNames) { - final var clientsHolder = s3PerProjectClientManager.getProjectClientsHolders().get(projectId); + final var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId); assertNotNull(clientsHolder); final Map s3ClientSettingsMap = clientsHolder.allClientSettings(); assertThat(s3ClientSettingsMap.keySet(), containsInAnyOrder(clientNames)); @@ -414,7 +430,7 @@ private void assertClientCredentials(ProjectId projectId, String clientName, Ama private void assertClientNotFound(ProjectId projectId, String clientName) { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, - () -> s3PerProjectClientManager.client(projectId, clientName) + () -> s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)) ); assertThat( e.getMessage(), @@ -465,6 +481,10 @@ private void removeProjectFromClusterState(ProjectId projectId) { } } + private RepositoryMetadata createRepositoryMetadata(String clientName) { + return new RepositoryMetadata("repo", S3Repository.TYPE, Settings.builder().put("client", clientName).build()); + } + private Map newProjectClientsSecrets(ProjectId projectId, String... clientNames) { s3SecretsIdGenerators.computeIfAbsent(projectId, ignored -> new AtomicInteger(0)).incrementAndGet(); final Map m = new HashMap<>(); diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java index 4d119f2353092..fcd2e60c4b7b5 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java @@ -64,7 +64,6 @@ public void testCachedClientsAreReleased() throws IOException { () -> Region.of("es-test-region") ); s3Service.start(); - assertNull(s3Service.getS3PerProjectClientManager()); final Settings settings = Settings.builder().put("endpoint", "http://first").build(); final RepositoryMetadata metadata1 = new RepositoryMetadata("first", "s3", settings); final RepositoryMetadata metadata2 = new RepositoryMetadata("second", "s3", settings); @@ -73,11 +72,11 @@ public void testCachedClientsAreReleased() throws IOException { assertSame(clientSettings, otherClientSettings); final AmazonS3Reference reference = s3Service.client(metadata1); reference.close(); - s3Service.doClose(); + s3Service.onBlobStoreClose(); final AmazonS3Reference referenceReloaded = s3Service.client(metadata1); assertNotSame(referenceReloaded, reference); referenceReloaded.close(); - s3Service.doClose(); + s3Service.onBlobStoreClose(); final S3ClientSettings clientSettingsReloaded = s3Service.settings(metadata1); assertNotSame(clientSettings, clientSettingsReloaded); s3Service.close(); @@ -126,7 +125,6 @@ public void testGetClientRegionFromSetting() { ) ) { s3Service.start(); - assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -165,7 +163,6 @@ public void testGetClientRegionFromEndpointSettingGuess() { ) ) { s3Service.start(); - assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -224,7 +221,6 @@ public void testGetClientRegionFromDefault() { ) ) { s3Service.start(); - assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); @@ -260,7 +256,6 @@ public void testGetClientRegionFallbackToUsEast1() { ) ) { s3Service.start(); - assertNull(s3Service.getS3PerProjectClientManager()); assertTrue(regionRequested.get()); final var clientName = randomBoolean() ? "default" : randomIdentifier(); From 4d7bd69741105ab0872ee9defe69d8f119c3eee0 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 14:47:19 +1000 Subject: [PATCH 10/24] no wait for async close --- .../repositories/s3/S3ClientsManager.java | 41 +----------- .../s3/S3ClientsManagerTests.java | 65 +------------------ 2 files changed, 3 insertions(+), 103 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index ae5b34749544f..fb29a7d587f45 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -9,8 +9,6 @@ package org.elasticsearch.repositories.s3; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterStateApplier; import org.elasticsearch.cluster.metadata.ProjectId; @@ -22,7 +20,6 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.IOUtils; -import org.elasticsearch.core.Nullable; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; @@ -33,9 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.function.UnaryOperator; @@ -53,8 +48,6 @@ public class S3ClientsManager implements ClusterStateApplier { private final Executor executor; // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. private final Map> clientsHolders; - // Listener for tracking ongoing async closing of obsolete clients. Updated only in the applier thread. - private volatile SubscribableListener clientsCloseListener = null; S3ClientsManager( Settings nodeSettings, @@ -142,31 +135,20 @@ public void applyClusterState(ClusterChangedEvent event) { } // Close stale clients asynchronously without blocking the applier thread if (clientsHoldersToClose.isEmpty() == false) { - final var currentClientsCloseListener = new SubscribableListener(); - final var previousClientsCloseListener = clientsCloseListener; - clientsCloseListener = currentClientsCloseListener; - if (previousClientsCloseListener != null && previousClientsCloseListener.isDone() == false) { - previousClientsCloseListener.addListener( - ActionListener.running(() -> closeClientsAsync(clientsHoldersToClose, currentClientsCloseListener)) - ); - } else { - closeClientsAsync(clientsHoldersToClose, currentClientsCloseListener); - } + closeClientsAsync(clientsHoldersToClose); } } - private void closeClientsAsync(List clientsHoldersToClose, ActionListener listener) { + private void closeClientsAsync(List clientsHoldersToClose) { executor.execute(new AbstractRunnable() { @Override protected void doRun() throws Exception { IOUtils.closeWhileHandlingException(clientsHoldersToClose); - listener.onResponse(null); } @Override public void onFailure(Exception e) { logger.warn("Failed to close s3 clients", e); - listener.onFailure(e); } }); } @@ -224,25 +206,6 @@ void releaseCachedClients(ProjectId projectId) { */ void close() { IOUtils.closeWhileHandlingException(clientsHolders.values()); - final var currentClientsCloseListener = clientsCloseListener; - if (currentClientsCloseListener != null && currentClientsCloseListener.isDone() == false) { - // Wait for async clients closing to be completed - final CountDownLatch latch = new CountDownLatch(1); - currentClientsCloseListener.addListener(ActionListener.running(latch::countDown)); - try { - if (latch.await(1, TimeUnit.MINUTES) == false) { - logger.warn("Waiting for async closing of s3 clients timed out"); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - - // visible for tests - @Nullable - SubscribableListener getClientsCloseListener() { - return clientsCloseListener; } private boolean newOrUpdated(ProjectId projectId, Map currentClientSettings) { diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index 4807f7b23f455..9b130d1bb0303 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -38,14 +38,12 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -75,14 +73,12 @@ public class S3ClientsManagerTests extends ESTestCase { private S3Service s3Service; private S3ClientsManager s3ClientsManager; private final AtomicReference clientRefsCloseLatchRef = new AtomicReference<>(); - private final AtomicBoolean closeInternalInvoked = new AtomicBoolean(false); @Override public void setUp() throws Exception { super.setUp(); s3SecretsIdGenerators = ConcurrentCollections.newConcurrentMap(); clientRefsCloseLatchRef.set(null); - closeInternalInvoked.set(false); clientNames = IntStream.range(0, between(2, 5)).mapToObj(i -> randomIdentifier() + "_" + i).toList(); final Settings.Builder builder = Settings.builder(); @@ -111,28 +107,7 @@ public void setUp() throws Exception { TestProjectResolvers.allProjects(), mock(ResourceWatcherService.class), () -> Region.of("es-test-region") - ) { - @Override - protected AmazonS3Reference buildClientReference(S3ClientSettings clientSettings) { - final var original = super.buildClientReference(clientSettings); - final var closeLatch = clientRefsCloseLatchRef.get(); - if (closeLatch == null) { - return original; - } - - original.decRef(); - final AmazonS3Reference proxy = new AmazonS3Reference(original.client(), DummySdkHttpClient.INSTANCE) { - @Override - protected void closeInternal() { - closeInternalInvoked.set(true); - safeAwait(closeLatch); - original.close(); - } - }; - proxy.mustIncRef(); - return proxy; - } - }; + ); s3Service.refreshAndClearCache(S3ClientSettings.load(settings)); s3ClientsManager = s3Service.getS3PerProjectClientManager(); assertNotNull(s3ClientsManager); @@ -145,8 +120,6 @@ public void tearDown() throws Exception { s3Service.close(); clusterService.close(); threadPool.close(); - final var clientsCloseListener = s3ClientsManager.getClientsCloseListener(); - assertTrue(clientsCloseListener == null || clientsCloseListener.isDone()); s3ClientsManager.getClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); } @@ -279,42 +252,6 @@ public void testClientsForMultipleProjects() throws InterruptedException { } } - public void testWaitForAsyncClientClose() throws Exception { - final CountDownLatch closeLatch = new CountDownLatch(1); - clientRefsCloseLatchRef.set(closeLatch); - - final List projectIds = randomList(1, 3, ESTestCase::randomUniqueProjectId); - final int iterations = between(3, 8); - - final List clientRefs = new ArrayList<>(); - for (int i = 0; i < iterations; i++) { - for (var projectId : projectIds) { - final List subsetOfClientNames = randomNonEmptySubsetOf(clientNames); - updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, subsetOfClientNames.toArray(String[]::new))); - subsetOfClientNames.forEach(clientName -> { - final var newClient = s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)); - clientRefs.add(newClient); - newClient.decRef(); - }); - if (randomBoolean() && randomBoolean()) { - removeProjectFromClusterState(projectId); - } - } - } - - final Thread thread = new Thread(() -> s3Service.close()); - thread.start(); - - assertBusy(() -> assertTrue(closeInternalInvoked.get())); - Thread.sleep(between(0, 100)); - assertFalse(s3ClientsManager.getClientsCloseListener().isDone()); - - closeLatch.countDown(); - assertTrue(thread.join(Duration.ofSeconds(10))); - assertTrue(s3ClientsManager.getClientsCloseListener().isDone()); - clientRefs.forEach(clientRef -> assertFalse(clientRef.hasReferences())); - } - public void testClusterAndProjectClients() { final ProjectId projectId = randomUniqueProjectId(); final String clientName = randomFrom(clientNames); From 6ce08284522471e3e15f1a65ef3d4b7b47ad7003 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 16:32:13 +1000 Subject: [PATCH 11/24] update --- .../repositories/s3/S3ClientsManager.java | 12 +++++++--- .../repositories/s3/S3Service.java | 24 ++++++++++--------- .../s3/S3ClientsManagerTests.java | 4 ++-- 3 files changed, 24 insertions(+), 16 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index fb29a7d587f45..f3bffe33ea51e 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -38,6 +38,13 @@ import static java.util.Collections.emptyMap; +/** + * The S3ClientsManager is responsible for managing Amazon S3 clients associated with either the cluster or projects. + * To use a single data structure for all clients, the cluster clients are stored against the default project-id. + * Note that the cluster level clients are created and refreshed based on the ReloadablePlugin interface, while + * the project level clients are created and refreshed by cluster state updates. All clients are released when + * the manager itself is closed. + */ public class S3ClientsManager implements ClusterStateApplier { private static final Logger logger = LogManager.getLogger(S3ClientsManager.class); @@ -188,8 +195,8 @@ AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetad } /** - * Similar to S3Service#releaseCachedClients but only clears the cache for the given project. - * All clients for the project are closed and will be recreated on next access, also similar to S3Service#releaseCachedClients + * Clears the cache for the given project (default project-id is for the cluster level clients). + * All clients for the project are closed and will be recreated on next access. */ void releaseCachedClients(ProjectId projectId) { final var old = clientsHolders.get(Objects.requireNonNull(projectId)); @@ -202,7 +209,6 @@ void releaseCachedClients(ProjectId projectId) { /** * Shutdown the manager by closing all clients holders. This is called when the node is shutting down. - * It attempts to wait (1 min) for any async client closing to complete. */ void close() { IOUtils.closeWhileHandlingException(clientsHolders.values()); diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 2e36c0ab2cf78..9053ad8bd4ff0 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -169,14 +169,19 @@ public AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { } /** - * Attempts to retrieve a project client from the project client manager. Throws if project-id or the client name does not exist. - * THe client maybe initialized lazily. - * Delegates to {@link #client(RepositoryMetadata)} when either of the followings is true: - * 1. Per-project client is disabled - * 2. Blobstore is cluster level (projectId = null) + * Attempts to retrieve either a cluster or project client from the client manager. Throws if project-id or + * the client name does not exist. The client maybe initialized lazily. + * @param projectId The project associated with the client, or null if the client is cluster level */ public AmazonS3Reference client(@Nullable ProjectId projectId, RepositoryMetadata repositoryMetadata) { - return s3ClientsManager.client(projectId == null ? ProjectId.DEFAULT : projectId, repositoryMetadata); + return s3ClientsManager.client(effectiveProjectId(projectId), repositoryMetadata); + } + + /** + * We use the default project-id for cluster level clients. + */ + ProjectId effectiveProjectId(@Nullable ProjectId projectId) { + return projectId == null ? ProjectId.DEFAULT : projectId; } // visible for tests @@ -418,13 +423,10 @@ public void onBlobStoreClose() { } /** - * Release all project clients. - * Delegates to {@link #onBlobStoreClose()} when either of the followings is true: - * 1. Per-project client is disabled - * 2. Blobstore is cluster level (projectId = null) + * Release clients for the specified project. */ public void onBlobStoreClose(@Nullable ProjectId projectId) { - s3ClientsManager.releaseCachedClients(projectId == null ? ProjectId.DEFAULT : projectId); + s3ClientsManager.releaseCachedClients(effectiveProjectId(projectId)); } @Override diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index 9b130d1bb0303..cdee4d9ecb5e1 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -295,8 +295,7 @@ public void testProjectClientsDisabled() { TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of("es-test-region") - ) { - }; + ); s3ServiceWithNoProjectSupport.refreshAndClearCache(S3ClientSettings.load(clusterService.getSettings())); s3ServiceWithNoProjectSupport.start(); assertNotNull(s3ServiceWithNoProjectSupport.getS3PerProjectClientManager()); @@ -317,6 +316,7 @@ public void testProjectClientsDisabled() { private Map> getClientsHoldersExcludeDefaultProject() { final var holders = s3ClientsManager.getClientsHolders(); + // Clients holder for the default project always exists assertThat(holders, hasKey(ProjectId.DEFAULT)); return holders.entrySet() .stream() From 616a770d0e27911482fd5a021830214a223f0d5d Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 16:35:31 +1000 Subject: [PATCH 12/24] tweak --- .../org/elasticsearch/repositories/s3/S3Service.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 9053ad8bd4ff0..29b5b02a4bdd7 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -184,8 +184,7 @@ ProjectId effectiveProjectId(@Nullable ProjectId projectId) { return projectId == null ? ProjectId.DEFAULT : projectId; } - // visible for tests - protected AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { + private AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); Releasable toRelease = httpClient::close; try { @@ -198,8 +197,13 @@ protected AmazonS3Reference buildClientReference(final S3ClientSettings clientSe } } + @FixForMultiProject(description = "can be removed once blobstore is project aware") S3ClientSettings settings(RepositoryMetadata repositoryMetadata) { - return s3ClientsManager.settingsForClient(ProjectId.DEFAULT, repositoryMetadata); + return settings(ProjectId.DEFAULT, repositoryMetadata); + } + + S3ClientSettings settings(@Nullable ProjectId projectId, RepositoryMetadata repositoryMetadata) { + return s3ClientsManager.settingsForClient(effectiveProjectId(projectId), repositoryMetadata); } // proxy for testing From 39fa18451caf1718d2a55f697be6e72c5baead8a Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 21 May 2025 16:53:10 +1000 Subject: [PATCH 13/24] tweak --- .../repositories/s3/S3ClientsManager.java | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index f3bffe33ea51e..e1683a374bff9 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -256,13 +256,6 @@ abstract class ClientsHolder implements Closeable { */ abstract ProjectId projectId(); - /** - * Determine the client name from the given repository metadata. - */ - String clientName(RepositoryMetadata repositoryMetadata) { - return S3Repository.CLIENT_NAME.get(repositoryMetadata.settings()); - } - /** * Similar to {@link #singleClientSettings(K)} but from the given repository metadata. */ @@ -290,12 +283,6 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { } final var settings = singleClientSettings(clientKey); - if (settings == null) { - throw new IllegalArgumentException( - "s3 client [" + clientName(repositoryMetadata) + "] does not exist for project [" + projectId() + "]" - ); - } - synchronized (this) { final var existing = clientsCache.get(clientKey); if (existing != null && existing.tryIncRef()) { @@ -303,7 +290,7 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { } if (closed.get()) { // Not adding a new client once the manager is closed since there won't be anything to close it - throw new IllegalStateException("client manager is closed"); + throw new IllegalStateException("clients holder is closed"); } // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. // The clearCache() will clear the newly added client. @@ -350,6 +337,7 @@ final class PerProjectClientsHolder extends ClientsHolder { private final Map clientSettings; PerProjectClientsHolder(ProjectId projectId, Map clientSettings) { + assert ProjectId.DEFAULT.equals(projectId) == false; this.projectId = projectId; this.clientSettings = clientSettings; } @@ -366,7 +354,11 @@ String clientKey(RepositoryMetadata repositoryMetadata) { @Override S3ClientSettings singleClientSettings(String clientKey) { - return clientSettings.get(clientKey); + final S3ClientSettings settings = clientSettings.get(clientKey); + if (settings == null) { + throw new IllegalArgumentException("s3 client [" + clientKey + "] does not exist for project [" + projectId + "]"); + } + return settings; } @Override From 922327bf511a6c9668005f0f98a049618d8129b8 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 22 May 2025 12:58:28 +1000 Subject: [PATCH 14/24] more assertion --- .../repositories/s3/S3ClientsManager.java | 2 +- .../repositories/s3/S3ClientsManagerTests.java | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index e1683a374bff9..09f5f34b42c05 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -290,7 +290,7 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { } if (closed.get()) { // Not adding a new client once the manager is closed since there won't be anything to close it - throw new IllegalStateException("clients holder is closed"); + throw new IllegalStateException("Project [" + projectId() + "] clients holder is closed"); } // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. // The clearCache() will clear the newly added client. diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index cdee4d9ecb5e1..8494c30f0a47c 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -152,7 +152,7 @@ public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); } - public void testClientsLifeCycleForSingleProject() { + public void testClientsLifeCycleForSingleProject() throws Exception { final ProjectId projectId = randomUniqueProjectId(); final String clientName = randomFrom(clientNames); final String anotherClientName = randomValueOtherThan(clientName, () -> randomFrom(clientNames)); @@ -197,6 +197,8 @@ public void testClientsLifeCycleForSingleProject() { antherClient.decRef(); } + final var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId); + // Remove project secrets if (randomBoolean()) { updateProjectInClusterState(projectId, Map.of()); @@ -204,6 +206,13 @@ public void testClientsLifeCycleForSingleProject() { removeProjectFromClusterState(projectId); } assertClientNotFound(projectId, clientName); + + assertBusy(() -> assertTrue(clientsHolder.isClosed())); + final var e = expectThrows( + IllegalStateException.class, + () -> clientsHolder.client(createRepositoryMetadata(randomFrom(clientName, anotherClientName))) + ); + assertThat(e.getMessage(), containsString("Project [" + projectId + "] clients holder is closed")); } public void testClientsForMultipleProjects() throws InterruptedException { From f4537ef5677c2085302f061bfa0db4cee63ef3fa Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 28 May 2025 10:01:19 +1000 Subject: [PATCH 15/24] no client creation after manager closing --- .../repositories/s3/S3ClientsManager.java | 13 ++++++++++- .../s3/S3ClientsManagerTests.java | 23 +++++++++++++++---- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index 09f5f34b42c05..cca865125cbd0 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -53,6 +53,7 @@ public class S3ClientsManager implements ClusterStateApplier { private final Settings nodeS3Settings; private final Function clientBuilder; private final Executor executor; + private final AtomicBoolean managerClosed = new AtomicBoolean(false); // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. private final Map> clientsHolders; @@ -211,7 +212,12 @@ void releaseCachedClients(ProjectId projectId) { * Shutdown the manager by closing all clients holders. This is called when the node is shutting down. */ void close() { - IOUtils.closeWhileHandlingException(clientsHolders.values()); + if (managerClosed.compareAndSet(false, true)) { + // Close all clients holders, they will close their cached clients. + // It's OK if a new clients holder is added concurrently or after this point because + // no new client will be created once the manager is closed, i.e. nothing to release. + IOUtils.closeWhileHandlingException(clientsHolders.values()); + } } private boolean newOrUpdated(ProjectId projectId, Map currentClientSettings) { @@ -292,6 +298,11 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { // Not adding a new client once the manager is closed since there won't be anything to close it throw new IllegalStateException("Project [" + projectId() + "] clients holder is closed"); } + if (managerClosed.get()) { + // This clients holder must be added after the manager is closed. It must have no cached clients. + assert clientsCache.isEmpty() : "expect empty cache, but got " + clientsCache; + throw new IllegalStateException("s3 clients manager is closed"); + } // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. // The clearCache() will clear the newly added client. final var newClientReference = clientBuilder.apply(settings); diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index 8494c30f0a47c..ef12fb5f79238 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -42,10 +42,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -72,13 +70,11 @@ public class S3ClientsManagerTests extends ESTestCase { private ClusterService clusterService; private S3Service s3Service; private S3ClientsManager s3ClientsManager; - private final AtomicReference clientRefsCloseLatchRef = new AtomicReference<>(); @Override public void setUp() throws Exception { super.setUp(); s3SecretsIdGenerators = ConcurrentCollections.newConcurrentMap(); - clientRefsCloseLatchRef.set(null); clientNames = IntStream.range(0, between(2, 5)).mapToObj(i -> randomIdentifier() + "_" + i).toList(); final Settings.Builder builder = Settings.builder(); @@ -296,6 +292,25 @@ public void testClusterAndProjectClients() { assertFalse(projectClient.hasReferences()); } + public void testClientsHolderAfterManagerClosed() { + final ProjectId projectId = randomUniqueProjectId(); + final String clientName = randomFrom(clientNames); + + s3ClientsManager.close(); + // New holder can be added after the manager is closed, but no actual client can be created + updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); + try (var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId)) { + assertNotNull(clientsHolder); + assertFalse(clientsHolder.isClosed()); + + final IllegalStateException e = expectThrows( + IllegalStateException.class, + () -> s3ClientsManager.client(projectId, createRepositoryMetadata(clientName)) + ); + assertThat(e.getMessage(), containsString("s3 clients manager is closed")); + } + } + public void testProjectClientsDisabled() { final var clusterService = spy(this.clusterService); final S3Service s3ServiceWithNoProjectSupport = new S3Service( From ab5f91163b991b99672d2bca2388dc8c6ec323c3 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 28 May 2025 19:36:01 +1000 Subject: [PATCH 16/24] remove dead code --- .../s3/S3ClientsManagerTests.java | 23 +++---------------- 1 file changed, 3 insertions(+), 20 deletions(-) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index ef12fb5f79238..c4e232100d1d7 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -10,13 +10,6 @@ package org.elasticsearch.repositories.s3; import joptsimple.internal.Strings; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.http.ExecutableHttpRequest; -import software.amazon.awssdk.http.HttpExecuteRequest; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; -import software.amazon.awssdk.regions.Region; - import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.ProjectId; @@ -35,6 +28,9 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; +import software.amazon.awssdk.regions.Region; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -463,17 +459,4 @@ private String projectClientAccessKey(ProjectId projectId, String clientName) { private String projectClientSecretKey(ProjectId projectId, String clientName) { return projectId + "_" + clientName + "_secret_key_" + s3SecretsIdGenerators.get(projectId).get(); } - - private static class DummySdkHttpClient implements SdkHttpClient { - - static final SdkHttpClient INSTANCE = new DummySdkHttpClient(); - - @Override - public ExecutableHttpRequest prepareRequest(HttpExecuteRequest request) { - return null; - } - - @Override - public void close() {} - } } From dc14ef75bc2cde9569322fc5a40967b2318ff978 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 28 May 2025 09:44:24 +0000 Subject: [PATCH 17/24] [CI] Auto commit changes from spotless --- .../repositories/s3/S3ClientsManagerTests.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index c4e232100d1d7..a5bf5472b7f7f 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -10,6 +10,10 @@ package org.elasticsearch.repositories.s3; import joptsimple.internal.Strings; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; +import software.amazon.awssdk.regions.Region; + import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.ProjectId; @@ -28,9 +32,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity; -import software.amazon.awssdk.regions.Region; import java.nio.charset.StandardCharsets; import java.time.Duration; From ff631de4ce5cd71ba4a1c4df9118ab53416abc2a Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 10:51:40 +1000 Subject: [PATCH 18/24] separate field for cluster clients --- .../repositories/s3/S3ClientsManager.java | 98 +++++++++++-------- .../repositories/s3/S3Service.java | 9 +- .../s3/S3ClientsManagerTests.java | 49 +++++----- 3 files changed, 87 insertions(+), 69 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index cca865125cbd0..5ab49d0926609 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -30,10 +30,10 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; -import java.util.function.UnaryOperator; import java.util.stream.Collectors; import static java.util.Collections.emptyMap; @@ -55,13 +55,14 @@ public class S3ClientsManager implements ClusterStateApplier { private final Executor executor; private final AtomicBoolean managerClosed = new AtomicBoolean(false); // A map of projectId to clients holder. Adding to and removing from the map happen only in the applier thread. - private final Map> clientsHolders; + private final Map perProjectClientsHolders; + private final ClusterClientsHolder clusterClientsHolder; S3ClientsManager( Settings nodeSettings, Function clientBuilder, - UnaryOperator>> clientsHoldersWrapper, - Executor executor + Executor executor, + boolean supportsMultipleProjects ) { this.nodeS3Settings = Settings.builder() .put(nodeSettings.getByPrefix(S3_SETTING_PREFIX), false) // not rely on any cluster scoped secrets @@ -69,17 +70,20 @@ public class S3ClientsManager implements ClusterStateApplier { .build(); this.clientBuilder = clientBuilder; this.executor = executor; - this.clientsHolders = clientsHoldersWrapper.apply(Map.of(ProjectId.DEFAULT, new ClusterClientsHolder())); + this.clusterClientsHolder = new ClusterClientsHolder(); + this.perProjectClientsHolders = supportsMultipleProjects ? new ConcurrentHashMap<>() : null; } @Override public void applyClusterState(ClusterChangedEvent event) { + assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; final Map currentProjects = event.state().metadata().projects(); final var updatedPerProjectClients = new HashMap(); final List clientsHoldersToClose = new ArrayList<>(); for (var project : currentProjects.values()) { - // Skip the default project, it is handled differently with the ReloadablePlugin interface + // Skip the default project, it is tracked separately with clusterClientsHolder and + // updated differently with the ReloadablePlugin interface if (ProjectId.DEFAULT.equals(project.id())) { continue; } @@ -87,10 +91,9 @@ public void applyClusterState(ClusterChangedEvent event) { // Project secrets can be null when node restarts. It may not have any s3 credentials if s3 is not in use. if (projectSecrets == null || projectSecrets.getSettingNames().stream().noneMatch(key -> key.startsWith("s3."))) { // Most likely there won't be any existing client, but attempt to remove it anyway just in case - final var removed = clientsHolders.remove(project.id()); + final var removed = perProjectClientsHolders.remove(project.id()); if (removed != null) { - assert removed instanceof PerProjectClientsHolder; - clientsHoldersToClose.add((PerProjectClientsHolder) removed); + clientsHoldersToClose.add(removed); } continue; } @@ -126,28 +129,27 @@ public void applyClusterState(ClusterChangedEvent event) { // Updated projects for (var projectId : updatedPerProjectClients.keySet()) { - final var old = clientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); + assert ProjectId.DEFAULT.equals(projectId) == false; + final var old = perProjectClientsHolders.put(projectId, updatedPerProjectClients.get(projectId)); if (old != null) { - assert old instanceof PerProjectClientsHolder; - clientsHoldersToClose.add((PerProjectClientsHolder) old); + clientsHoldersToClose.add(old); } } - // removed projects - for (var projectId : clientsHolders.keySet()) { + // Removed projects + for (var projectId : perProjectClientsHolders.keySet()) { + assert ProjectId.DEFAULT.equals(projectId) == false; if (currentProjects.containsKey(projectId) == false) { - assert ProjectId.DEFAULT.equals(projectId) == false; - final var removed = clientsHolders.remove(projectId); - assert removed instanceof PerProjectClientsHolder; - clientsHoldersToClose.add((PerProjectClientsHolder) removed); + final var removed = perProjectClientsHolders.remove(projectId); + clientsHoldersToClose.add(removed); } } // Close stale clients asynchronously without blocking the applier thread if (clientsHoldersToClose.isEmpty() == false) { - closeClientsAsync(clientsHoldersToClose); + closePerProjectClientsAsync(clientsHoldersToClose); } } - private void closeClientsAsync(List clientsHoldersToClose) { + private void closePerProjectClientsAsync(List clientsHoldersToClose) { executor.execute(new AbstractRunnable() { @Override protected void doRun() throws Exception { @@ -162,34 +164,45 @@ public void onFailure(Exception e) { } // visible for tests - Map> getClientsHolders() { - return Map.copyOf(clientsHolders); + ClusterClientsHolder getClusterClientsHolder() { + return clusterClientsHolder; + } + + // visible for tests + Map getPerProjectClientsHolders() { + return perProjectClientsHolders == null ? null : Map.copyOf(perProjectClientsHolders); + } + + // visible for tests + boolean isManagerClosed() { + return managerClosed.get(); } void refreshAndClearCacheForClusterClients(Map clientsSettings) { - final var clientsHolder = clientsHolders.get(ProjectId.DEFAULT); - if (clientsHolder instanceof ClusterClientsHolder clusterClientsHolder) { - clusterClientsHolder.refreshAndClearCache(clientsSettings); - } else { - final String message = "expect cluster clients holder, got " + clientsHolder; - assert false : message; - throw new IllegalStateException(message); - } + clusterClientsHolder.refreshAndClearCache(clientsSettings); } S3ClientSettings settingsForClient(ProjectId projectId, RepositoryMetadata repositoryMetadata) { - final var clientsHolder = clientsHolders.get(Objects.requireNonNull(projectId)); + if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { + return clusterClientsHolder.singleClientSettings(repositoryMetadata); + } + + assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; + final var clientsHolder = perProjectClientsHolders.get(projectId); if (clientsHolder == null) { - assert ProjectId.DEFAULT.equals(projectId) == false; throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); } return clientsHolder.singleClientSettings(repositoryMetadata); } AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetadata) { - final var clientsHolder = clientsHolders.get(Objects.requireNonNull(projectId)); + if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { + return clusterClientsHolder.client(repositoryMetadata); + } + + assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; + final var clientsHolder = perProjectClientsHolders.get(projectId); if (clientsHolder == null) { - assert ProjectId.DEFAULT.equals(projectId) == false; throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); } return clientsHolder.client(repositoryMetadata); @@ -200,11 +213,15 @@ AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetad * All clients for the project are closed and will be recreated on next access. */ void releaseCachedClients(ProjectId projectId) { - final var old = clientsHolders.get(Objects.requireNonNull(projectId)); + if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { + clusterClientsHolder.clearCache(); + return; + } + + assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; + final var old = perProjectClientsHolders.get(projectId); if (old != null) { old.clearCache(); - } else { - assert ProjectId.DEFAULT.equals(projectId) == false; } } @@ -216,12 +233,15 @@ void close() { // Close all clients holders, they will close their cached clients. // It's OK if a new clients holder is added concurrently or after this point because // no new client will be created once the manager is closed, i.e. nothing to release. - IOUtils.closeWhileHandlingException(clientsHolders.values()); + if (perProjectClientsHolders != null) { + IOUtils.closeWhileHandlingException(perProjectClientsHolders.values()); + } + IOUtils.closeWhileHandlingException(clusterClientsHolder); } } private boolean newOrUpdated(ProjectId projectId, Map currentClientSettings) { - final var old = clientsHolders.get(projectId); + final var old = perProjectClientsHolders.get(projectId); if (old == null) { return true; } diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 29b5b02a4bdd7..b617b7b797769 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -71,10 +71,8 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Consumer; import java.util.function.Supplier; -import java.util.function.UnaryOperator; import static software.amazon.awssdk.core.SdkSystemSetting.AWS_ROLE_ARN; import static software.amazon.awssdk.core.SdkSystemSetting.AWS_ROLE_SESSION_NAME; @@ -136,8 +134,8 @@ class S3Service extends AbstractLifecycleComponent { s3ClientsManager = new S3ClientsManager( nodeSettings, this::buildClientReference, - projectResolver.supportsMultipleProjects() ? ConcurrentHashMap::new : UnaryOperator.identity(), - clusterService.threadPool().generic() + clusterService.threadPool().generic(), + projectResolver.supportsMultipleProjects() ); if (projectResolver.supportsMultipleProjects()) { clusterService.addHighPriorityApplier(s3ClientsManager); @@ -145,7 +143,7 @@ class S3Service extends AbstractLifecycleComponent { } // visible to tests - S3ClientsManager getS3PerProjectClientManager() { + S3ClientsManager getS3ClientsManager() { return s3ClientsManager; } @@ -428,6 +426,7 @@ public void onBlobStoreClose() { /** * Release clients for the specified project. + * @param projectId The project associated with the client, or null if the client is cluster level */ public void onBlobStoreClose(@Nullable ProjectId projectId) { s3ClientsManager.releaseCachedClients(effectiveProjectId(projectId)); diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index a5bf5472b7f7f..d5c685a454880 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -97,13 +97,14 @@ public void setUp() throws Exception { s3Service = new S3Service( mock(Environment.class), clusterService, - TestProjectResolvers.allProjects(), + TestProjectResolvers.allProjects(), // with multiple projects support mock(ResourceWatcherService.class), () -> Region.of("es-test-region") ); s3Service.refreshAndClearCache(S3ClientSettings.load(settings)); - s3ClientsManager = s3Service.getS3PerProjectClientManager(); - assertNotNull(s3ClientsManager); + s3ClientsManager = s3Service.getS3ClientsManager(); + assertThat(s3ClientsManager.getClusterClientsHolder().allClientSettings(), equalTo(clusterClientsSettings)); + assertNotNull(s3ClientsManager.getPerProjectClientsHolders()); s3Service.start(); } @@ -113,11 +114,13 @@ public void tearDown() throws Exception { s3Service.close(); clusterService.close(); threadPool.close(); - s3ClientsManager.getClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); + assertTrue(s3ClientsManager.isManagerClosed()); + s3ClientsManager.getPerProjectClientsHolders().forEach((projectId, clientsHolder) -> assertTrue(clientsHolder.isClosed())); + assertTrue(s3ClientsManager.getClusterClientsHolder().isClosed()); } public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { - assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); + assertThat(s3ClientsManager.getPerProjectClientsHolders(), anEmptyMap()); final ProjectId projectId = randomUniqueProjectId(); // No project secrets at all @@ -125,7 +128,7 @@ public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { clusterService, ClusterState.builder(clusterService.state()).putProjectMetadata(ProjectMetadata.builder(projectId)).build() ); - assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); + assertThat(s3ClientsManager.getPerProjectClientsHolders(), anEmptyMap()); // Project secrets but no s3 credentials final var mockSecureSettings = new MockSecureSettings(); @@ -142,7 +145,7 @@ public void testDoesNotCreateClientWhenSecretsAreNotConfigured() { ) .build() ); - assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); + assertThat(s3ClientsManager.getPerProjectClientsHolders(), anEmptyMap()); } public void testClientsLifeCycleForSingleProject() throws Exception { @@ -190,9 +193,9 @@ public void testClientsLifeCycleForSingleProject() throws Exception { antherClient.decRef(); } - final var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId); + final var clientsHolder = s3ClientsManager.getPerProjectClientsHolders().get(projectId); - // Remove project secrets + // Remove project secrets or the entire project if (randomBoolean()) { updateProjectInClusterState(projectId, Map.of()); } else { @@ -242,7 +245,7 @@ public void testClientsForMultipleProjects() throws InterruptedException { } else { removeProjectFromClusterState(projectId); } - assertThat(getClientsHoldersExcludeDefaultProject(), not(hasKey(projectId))); + assertThat(s3ClientsManager.getPerProjectClientsHolders(), not(hasKey(projectId))); clientNames.forEach(clientName -> assertClientNotFound(projectId, clientName)); } } @@ -268,9 +271,9 @@ public void testClusterAndProjectClients() { Settings.builder().put("client", clientName).build() ); - final AmazonS3Reference clusterClient = s3Service.client(null, repositoryMetadata); + final AmazonS3Reference clusterClient = s3Service.client(projectIdForClusterClient(), repositoryMetadata); if (configureProjectClientsFirst == false) { - assertThat(getClientsHoldersExcludeDefaultProject(), anEmptyMap()); + assertThat(s3ClientsManager.getPerProjectClientsHolders(), anEmptyMap()); } clusterClient.decRef(); @@ -281,10 +284,12 @@ public void testClusterAndProjectClients() { assertThat(projectClient, not(sameInstance(clusterClient))); projectClient.decRef(); - s3Service.onBlobStoreClose(null); + // Release the cluster client + s3Service.onBlobStoreClose(projectIdForClusterClient()); assertFalse(clusterClient.hasReferences()); assertTrue(projectClient.hasReferences()); + // Release the project client s3Service.onBlobStoreClose(projectId); assertFalse(projectClient.hasReferences()); } @@ -296,7 +301,7 @@ public void testClientsHolderAfterManagerClosed() { s3ClientsManager.close(); // New holder can be added after the manager is closed, but no actual client can be created updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); - try (var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId)) { + try (var clientsHolder = s3ClientsManager.getPerProjectClientsHolders().get(projectId)) { assertNotNull(clientsHolder); assertFalse(clientsHolder.isClosed()); @@ -319,8 +324,8 @@ public void testProjectClientsDisabled() { ); s3ServiceWithNoProjectSupport.refreshAndClearCache(S3ClientSettings.load(clusterService.getSettings())); s3ServiceWithNoProjectSupport.start(); - assertNotNull(s3ServiceWithNoProjectSupport.getS3PerProjectClientManager()); verify(clusterService, never()).addHighPriorityApplier(any()); + assertNull(s3ServiceWithNoProjectSupport.getS3ClientsManager().getPerProjectClientsHolders()); // Cluster client still works final String clientName = randomFrom(clientNames); @@ -329,24 +334,18 @@ public void testProjectClientsDisabled() { "s3", Settings.builder().put("client", clientName).build() ); - final AmazonS3Reference clientRef = s3ServiceWithNoProjectSupport.client(ProjectId.DEFAULT, repositoryMetadata); + final AmazonS3Reference clientRef = s3ServiceWithNoProjectSupport.client(projectIdForClusterClient(), repositoryMetadata); clientRef.decRef(); s3ServiceWithNoProjectSupport.close(); assertFalse(clientRef.hasReferences()); } - private Map> getClientsHoldersExcludeDefaultProject() { - final var holders = s3ClientsManager.getClientsHolders(); - // Clients holder for the default project always exists - assertThat(holders, hasKey(ProjectId.DEFAULT)); - return holders.entrySet() - .stream() - .filter(entry -> entry.getKey() != ProjectId.DEFAULT) - .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + private ProjectId projectIdForClusterClient() { + return randomBoolean() ? ProjectId.DEFAULT : null; } private void assertProjectClientSettings(ProjectId projectId, String... clientNames) { - final var clientsHolder = s3ClientsManager.getClientsHolders().get(projectId); + final var clientsHolder = s3ClientsManager.getPerProjectClientsHolders().get(projectId); assertNotNull(clientsHolder); final Map s3ClientSettingsMap = clientsHolder.allClientSettings(); assertThat(s3ClientSettingsMap.keySet(), containsInAnyOrder(clientNames)); From 396210651e01f8248a12b0db8eb0b356230a4c26 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 10:53:09 +1000 Subject: [PATCH 19/24] assert no double close --- .../org/elasticsearch/repositories/s3/S3ClientsManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index 5ab49d0926609..4c0d7525b270e 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -237,6 +237,8 @@ void close() { IOUtils.closeWhileHandlingException(perProjectClientsHolders.values()); } IOUtils.closeWhileHandlingException(clusterClientsHolder); + } else { + assert false : "attempting to close s3 clients manager multiple times"; } } From 6cc05c62235c6298bd3cc08be29efe898b7012fe Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 11:01:45 +1000 Subject: [PATCH 20/24] use lightweight thread pool --- .../s3/AwsS3ServiceImplTests.java | 6 ++-- .../s3/S3BlobContainerRetriesTests.java | 11 ++++---- .../s3/S3ClientSettingsTests.java | 5 ++-- .../repositories/s3/S3RepositoryTests.java | 18 ++---------- .../repositories/s3/S3ServiceTests.java | 28 +++++-------------- 5 files changed, 18 insertions(+), 50 deletions(-) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java index e38da8c6bb7b2..e34ed727c0fc4 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/AwsS3ServiceImplTests.java @@ -25,10 +25,10 @@ import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -235,10 +235,9 @@ private void assertEndpoint(Settings repositorySettings, Settings settings, Stri public void testEndPointAndRegionOverrides() throws IOException { try ( - TestThreadPool threadPool = new TestThreadPool(getTestName()); S3Service s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of("es-test-region") @@ -253,7 +252,6 @@ public void testEndPointAndRegionOverrides() throws IOException { assertEquals("es-test-region", reference.client().serviceClientConfiguration().region().toString()); reference.close(); - s3Service.doClose(); } } diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 1b981f7c8a9a9..1e9982a8146d9 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -58,7 +58,6 @@ import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.hamcrest.Matcher; import org.junit.After; @@ -122,7 +121,6 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTestCase { private static final int MAX_NUMBER_SNAPSHOT_DELETE_RETRIES = 10; - private TestThreadPool threadPool; private S3Service service; private volatile boolean shouldErrorOnDns; private RecordingMeterRegistry recordingMeterRegistry; @@ -130,10 +128,9 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes @Before public void setUp() throws Exception { shouldErrorOnDns = false; - threadPool = new TestThreadPool(getTestClass().getName()); service = new S3Service( Mockito.mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, Mockito.mock(ResourceWatcherService.class), () -> null @@ -172,7 +169,6 @@ Optional getConnectionAcquisitionTimeout() { @After public void tearDown() throws Exception { IOUtils.close(service); - threadPool.close(); super.tearDown(); } @@ -1320,7 +1316,10 @@ public void testRetryOn403InStateless() { service = new S3Service( Mockito.mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool, Settings.builder().put(STATELESS_ENABLED_SETTING_NAME, "true").build()), + ClusterServiceUtils.createClusterService( + new DeterministicTaskQueue().getThreadPool(), + Settings.builder().put(STATELESS_ENABLED_SETTING_NAME, "true").build() + ), TestProjectResolvers.DEFAULT_PROJECT_ONLY, Mockito.mock(ResourceWatcherService.class), () -> null diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java index 2f161f1576195..26b9aab5569e1 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientSettingsTests.java @@ -16,10 +16,10 @@ import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.mockito.Mockito; @@ -185,10 +185,9 @@ public void testRegionCanBeSet() { assertThat(settings.get("other").region, is(randomRegion)); try ( - TestThreadPool threadPool = new TestThreadPool(getTestName()); var s3Service = new S3Service( Mockito.mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, Mockito.mock(ResourceWatcherService.class), () -> null diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java index bf5b72a3203aa..e2abf78e4e8ab 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java @@ -23,13 +23,13 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.env.Environment; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.hamcrest.Matchers; @@ -46,20 +46,6 @@ public class S3RepositoryTests extends ESTestCase { - private TestThreadPool threadPool; - - @Override - public void setUp() throws Exception { - super.setUp(); - threadPool = new TestThreadPool(getTestName()); - } - - @Override - public void tearDown() throws Exception { - super.tearDown(); - threadPool.close(); - } - private static class DummyS3Client implements S3Client { @Override @@ -185,7 +171,7 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { NamedXContentRegistry.EMPTY, new DummyS3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class) ), diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java index fcd2e60c4b7b5..6b8a1bc7a3d59 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ServiceTests.java @@ -23,13 +23,13 @@ import org.elasticsearch.cluster.project.TestProjectResolvers; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.env.Environment; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import java.io.IOException; @@ -41,24 +41,10 @@ public class S3ServiceTests extends ESTestCase { - private TestThreadPool threadPool; - - @Override - public void setUp() throws Exception { - super.setUp(); - threadPool = new TestThreadPool(getTestName()); - } - - @Override - public void tearDown() throws Exception { - super.tearDown(); - threadPool.close(); - } - public void testCachedClientsAreReleased() throws IOException { final S3Service s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of("es-test-region") @@ -115,7 +101,7 @@ public void testGetClientRegionFromSetting() { try ( var s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> { @@ -153,7 +139,7 @@ public void testGetClientRegionFromEndpointSettingGuess() { try ( var s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> { @@ -211,7 +197,7 @@ public void testGetClientRegionFromDefault() { try ( var s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> { @@ -246,7 +232,7 @@ public void testGetClientRegionFallbackToUsEast1() { try ( var s3Service = new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> { @@ -317,7 +303,7 @@ public void testEndpointOverrideSchemeUsesHttpIfHttpProtocolSpecified() { private URI getEndpointUri(Settings.Builder settings, String clientName) { return new S3Service( mock(Environment.class), - ClusterServiceUtils.createClusterService(threadPool), + ClusterServiceUtils.createClusterService(new DeterministicTaskQueue().getThreadPool()), TestProjectResolvers.DEFAULT_PROJECT_ONLY, mock(ResourceWatcherService.class), () -> Region.of(randomIdentifier()) From ef5631b31c554747843157d68831c9a9c1fedf1c Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 11:33:29 +1000 Subject: [PATCH 21/24] tweak --- .../repositories/s3/S3ClientsManager.java | 49 +++++++++---------- 1 file changed, 22 insertions(+), 27 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index 4c0d7525b270e..9cc3de0335c89 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -20,6 +20,7 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Nullable; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; @@ -183,29 +184,11 @@ void refreshAndClearCacheForClusterClients(Map clients } S3ClientSettings settingsForClient(ProjectId projectId, RepositoryMetadata repositoryMetadata) { - if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { - return clusterClientsHolder.singleClientSettings(repositoryMetadata); - } - - assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; - final var clientsHolder = perProjectClientsHolders.get(projectId); - if (clientsHolder == null) { - throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); - } - return clientsHolder.singleClientSettings(repositoryMetadata); + return getClientsHolderSafe(projectId).singleClientSettings(repositoryMetadata); } AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetadata) { - if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { - return clusterClientsHolder.client(repositoryMetadata); - } - - assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; - final var clientsHolder = perProjectClientsHolders.get(projectId); - if (clientsHolder == null) { - throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); - } - return clientsHolder.client(repositoryMetadata); + return getClientsHolderSafe(projectId).client(repositoryMetadata); } /** @@ -213,18 +196,30 @@ AmazonS3Reference client(ProjectId projectId, RepositoryMetadata repositoryMetad * All clients for the project are closed and will be recreated on next access. */ void releaseCachedClients(ProjectId projectId) { - if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { - clusterClientsHolder.clearCache(); - return; - } - - assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; - final var old = perProjectClientsHolders.get(projectId); + final var old = getClientsHolder(projectId); if (old != null) { old.clearCache(); } } + private ClientsHolder getClientsHolderSafe(ProjectId projectId) { + final var clientsHolder = getClientsHolder(projectId); + if (clientsHolder == null) { + throw new IllegalArgumentException("no s3 client is configured for project [" + projectId + "]"); + } + return clientsHolder; + } + + @Nullable + private ClientsHolder getClientsHolder(ProjectId projectId) { + if (ProjectId.DEFAULT.equals(Objects.requireNonNull(projectId))) { + return clusterClientsHolder; + } else { + assert perProjectClientsHolders != null : "expect per-project clients holders to be non-null"; + return perProjectClientsHolders.get(projectId); + } + } + /** * Shutdown the manager by closing all clients holders. This is called when the node is shutting down. */ From 9066af39e515b80cbde6029f8902ce252d942580 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 11:41:08 +1000 Subject: [PATCH 22/24] comments --- .../org/elasticsearch/repositories/s3/S3ClientsManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java index 9cc3de0335c89..5222835d34e18 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3ClientsManager.java @@ -312,7 +312,7 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { return existing; } if (closed.get()) { - // Not adding a new client once the manager is closed since there won't be anything to close it + // Not adding a new client once the clients holder is closed since there won't be anything to close it throw new IllegalStateException("Project [" + projectId() + "] clients holder is closed"); } if (managerClosed.get()) { @@ -321,7 +321,7 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { throw new IllegalStateException("s3 clients manager is closed"); } // The close() method maybe called after we checked it, it is ok since we are already inside the synchronized block. - // The clearCache() will clear the newly added client. + // The close method calls clearCache() which will clear the newly added client. final var newClientReference = clientBuilder.apply(settings); clientsCache = Maps.copyMapWithAddedEntry(clientsCache, clientKey, newClientReference); return newClientReference; @@ -329,7 +329,7 @@ final AmazonS3Reference client(RepositoryMetadata repositoryMetadata) { } /** - * Clear the cache by closing and clear out all clients. Subsequent {@link #client(RepositoryMetadata)} calls will recreate + * Clear the cache by closing and clearing out all clients. Subsequent {@link #client(RepositoryMetadata)} calls will recreate * the clients and populate the cache again. */ final synchronized void clearCache() { From 4048064cec2aaee8f693eab2c71b8998dc6630ec Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 12:35:41 +1000 Subject: [PATCH 23/24] fix test --- .../elasticsearch/repositories/s3/S3ClientsManagerTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java index d5c685a454880..f03723d95a2fc 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3ClientsManagerTests.java @@ -298,7 +298,8 @@ public void testClientsHolderAfterManagerClosed() { final ProjectId projectId = randomUniqueProjectId(); final String clientName = randomFrom(clientNames); - s3ClientsManager.close(); + s3Service.close(); + assertTrue(s3ClientsManager.isManagerClosed()); // New holder can be added after the manager is closed, but no actual client can be created updateProjectInClusterState(projectId, newProjectClientsSecrets(projectId, clientName)); try (var clientsHolder = s3ClientsManager.getPerProjectClientsHolders().get(projectId)) { From e1862dd150073b01e923187c18ace2d69c7e8d65 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 29 May 2025 22:24:18 +1000 Subject: [PATCH 24/24] comment --- .../main/java/org/elasticsearch/repositories/s3/S3Service.java | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index b617b7b797769..65ad20ce34ab0 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -182,6 +182,7 @@ ProjectId effectiveProjectId(@Nullable ProjectId projectId) { return projectId == null ? ProjectId.DEFAULT : projectId; } + // TODO: consider moving client building into S3ClientsManager private AmazonS3Reference buildClientReference(final S3ClientSettings clientSettings) { final SdkHttpClient httpClient = buildHttpClient(clientSettings, getCustomDnsResolver()); Releasable toRelease = httpClient::close;