xiaoxuandev commented on code in PR #16129:
URL: https://github.com/apache/iceberg/pull/16129#discussion_r3156742160
##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -1169,6 +1213,126 @@ public <T extends S3ClientBuilder> void
applyUserAgentConfigurations(T builder)
.build());
}
+ /**
+ * Applies a shared ScheduledExecutorService to the S3Client to prevent
unbounded thread
+ * accumulation.
+ */
+ public <T extends S3ClientBuilder> void applySharedScheduledExecutor(T
builder) {
+ if (!isSharedScheduledExecutorEnabled) {
+ return;
+ }
+
+ // Without this, each S3Client creates its own sdk-ScheduledExecutor
threads that are never
+ // shut down when the client becomes unreachable (threads are GC roots).
+ // https://github.com/apache/iceberg/issues/15898
+ ClientOverrideConfiguration.Builder configBuilder =
+ null != builder.overrideConfiguration()
+ ? builder.overrideConfiguration().toBuilder()
+ : ClientOverrideConfiguration.builder();
+ builder.overrideConfiguration(
+ configBuilder
+ .scheduledExecutorService(
+
SharedScheduledExecutorHolder.get(sharedScheduledExecutorPoolSize))
+ .build());
+ }
+
+ /**
+ * Lazy holder for the shared ScheduledExecutorService. Uses daemon threads
so they don't prevent
+ * JVM shutdown.
+ *
+ * <p>The returned executor is wrapped to prevent shutdown by the AWS SDK.
When S3Client.close()
+ * is called, the SDK's AttributeMap.closeIfPossible() calls shutdown() on
any ExecutorService
+ * stored in the client configuration. The wrapper makes
shutdown/shutdownNow no-ops so the shared
+ * executor remains usable across multiple S3 client lifecycles.
+ */
+ private static final class SharedScheduledExecutorHolder {
+ private static final AtomicInteger THREAD_COUNT = new AtomicInteger(0);
+ private static ScheduledExecutorService instance;
+
+ static synchronized ScheduledExecutorService get(int poolSize) {
+ if (instance == null) {
+ ScheduledExecutorService delegate =
+ Executors.newScheduledThreadPool(
+ poolSize,
+ runnable -> {
+ Thread thread = new Thread(runnable);
+ thread.setName("iceberg-shared-sdk-executor-" +
THREAD_COUNT.getAndIncrement());
+ thread.setDaemon(true);
+ return thread;
+ });
+ instance = new NonClosingScheduledExecutorService(delegate);
+ }
+ return instance;
+ }
+ }
+
+ /**
+ * A wrapper around a ScheduledExecutorService that prevents shutdown. This
is necessary because
+ * the AWS SDK v2 calls ExecutorService.shutdown() on client close via
+ * AttributeMap.closeIfPossible().
+ */
+ private static class NonClosingScheduledExecutorService extends
AbstractExecutorService
+ implements ScheduledExecutorService {
+ private final ScheduledExecutorService delegate;
+
+ NonClosingScheduledExecutorService(ScheduledExecutorService delegate) {
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void shutdown() {
+ // no-op: prevent SDK from shutting down the shared executor
+ }
+
+ @Override
+ public List<Runnable> shutdownNow() {
Review Comment:
added, thanks!
##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -506,6 +514,28 @@ public class S3FileIOProperties implements Serializable {
public static final boolean
S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT = true;
+ /**
+ * Controls whether a shared ScheduledExecutorService is used across S3
clients to prevent
+ * unbounded thread accumulation. When enabled, all S3 clients share a
single executor instead of
+ * each creating its own sdk-ScheduledExecutor threads that are never shut
down.
+ */
+ public static final String S3_SHARED_SCHEDULED_EXECUTOR_ENABLED =
+ "s3.shared-scheduled-executor.enabled";
+
+ public static final boolean S3_SHARED_SCHEDULED_EXECUTOR_ENABLED_DEFAULT =
true;
+
+ /**
+ * Pool size for the shared ScheduledExecutorService. Matches the AWS SDK v2
default pool size.
Review Comment:
updated, thanks!
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]