igorcalabria opened a new issue, #8601: URL: https://github.com/apache/iceberg/issues/8601
### Apache Iceberg version 1.3.1 (latest release) ### Query engine Spark ### Please describe the bug 🐞 Hi, we're using iceberg on a long running spark sql server and after upgrading from 1.2 to 1.3.1 we noticed that eventually the server starts throwing `java.lang.IllegalStateException: Connection pool shut down` on s3's connection pool. Full stack trace: ``` 2023-09-19T21:59:41+01:00 java.lang.IllegalStateException: Connection pool shut down 2023-09-19T21:59:41+01:00 at org.apache.http.util.Asserts.check(Asserts.java:34) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.conn.PoolingHttpClientConnectionManager.requestConnection(PoolingHttpClientConnectionManager.java:269) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$DelegatingHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:75) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$InstrumentedHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:57) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:176) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83) 2023-09-19T21:59:41+01:00 at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.internal.impl.ApacheSdkHttpClient.execute(ApacheSdkHttpClient.java:72) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.ApacheHttpClient.execute(ApacheHttpClient.java:254) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.ApacheHttpClient.access$500(ApacheHttpClient.java:104) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:231) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:228) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:63) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:77) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:56) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:39) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:73) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.DefaultStsClient.assumeRoleWithWebIdentity(DefaultStsClient.java:755) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.getUpdatedCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:73) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.updateSessionCredentials(StsCredentialsProvider.java:88) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.utils.cache.CachedSupplier.lambda$jitteredPrefetchValueSupplier$3(CachedSupplier.java:284) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.utils.cache.CachedSupplier$PrefetchStrategy.fetch(CachedSupplier.java:420) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.utils.cache.CachedSupplier.refreshCache(CachedSupplier.java:199) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.utils.cache.CachedSupplier.get(CachedSupplier.java:128) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.resolveCredentials(StsCredentialsProvider.java:99) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.resolveCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:44) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.sts.internal.StsWebIdentityCredentialsProviderFactory$StsWebIdentityCredentialsProvider.resolveCredentials(StsWebIdentityCredentialsProviderFactory.java:93) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider.resolveCredentials(WebIdentityTokenFileCredentialsProvider.java:113) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.resolveCredentials(AwsCredentialsProviderChain.java:90) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.auth.credentials.internal.LazyAwsCredentialsProvider.resolveCredentials(LazyAwsCredentialsProvider.java:45) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.resolveCredentials(DefaultCredentialsProvider.java:126) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.util.MetricUtils.measureDuration(MetricUtils.java:50) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.resolveCredentials(AwsCredentialsAuthorizationStrategy.java:100) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.addCredentialsToExecutionAttributes(AwsCredentialsAuthorizationStrategy.java:77) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.internal.AwsExecutionContextBuilder.invokeInterceptorsAndCreateExecutionContext(AwsExecutionContextBuilder.java:123) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.invokeInterceptorsAndCreateExecutionContext(AwsSyncClientHandler.java:69) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:64) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:62) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:63) 2023-09-19T21:59:41+01:00 at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:4478) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:192) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.aws.s3.S3InputStream.positionStream(S3InputStream.java:177) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.aws.s3.S3InputStream.read(S3InputStream.java:107) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1655) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1083) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3666) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:273) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:266) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$0(BaseMetastoreTableOperations.java:189) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$1(BaseMetastoreTableOperations.java:208) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:208) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:185) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:180) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.hive.HiveTableOperations.doRefresh(HiveTableOperations.java:178) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:97) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:80) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.BaseMetastoreCatalog.loadTable(BaseMetastoreCatalog.java:47) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2406) 2023-09-19T21:59:41+01:00 at java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1908) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2404) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2387) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:108) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.get(LocalManualCache.java:62) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.CachingCatalog.loadTable(CachingCatalog.java:166) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.spark.SparkCatalog.load(SparkCatalog.java:642) 2023-09-19T21:59:41+01:00 at org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:160) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.connector.catalog.CatalogV2Util$.loadTable(CatalogV2Util.scala:311) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.$anonfun$lookupTableOrView$2(Analyzer.scala:1138) 2023-09-19T21:59:41+01:00 at scala.Option.orElse(Option.scala:447) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$lookupTableOrView(Analyzer.scala:1136) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$13.applyOrElse(Analyzer.scala:1094) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$13.applyOrElse(Analyzer.scala:1032) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:30) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$2(AnalysisHelper.scala:135) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.DescribeRelation.mapChildren(v2Commands.scala:494) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:135) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:30) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:1032) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:991) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:211) 2023-09-19T21:59:41+01:00 at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) 2023-09-19T21:59:41+01:00 at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) 2023-09-19T21:59:41+01:00 at scala.collection.immutable.List.foldLeft(List.scala:91) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200) 2023-09-19T21:59:41+01:00 at scala.collection.immutable.List.foreach(List.scala:431) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:227) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:223) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:172) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:223) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:187) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:208) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:207) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:76) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:185) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:510) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:185) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:184) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:76) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:74) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:66) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:99) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:622) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:617) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:651) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:291) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.$anonfun$run$2(SparkExecuteStatementOperation.scala:230) 2023-09-19T21:59:41+01:00 at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties(SparkOperation.scala:79) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties$(SparkOperation.scala:63) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withLocalProperties(SparkExecuteStatementOperation.scala:43) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:230) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:225) 2023-09-19T21:59:41+01:00 at java.base/java.security.AccessController.doPrivileged(Native Method) 2023-09-19T21:59:41+01:00 at java.base/javax.security.auth.Subject.doAs(Subject.java:423) 2023-09-19T21:59:41+01:00 at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878) 2023-09-19T21:59:41+01:00 at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2.run(SparkExecuteStatementOperation.scala:239) 2023-09-19T21:59:41+01:00 at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) 2023-09-19T21:59:41+01:00 at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) 2023-09-19T21:59:41+01:00 at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 2023-09-19T21:59:41+01:00 at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 2023-09-19T21:59:41+01:00 at java.base/java.lang.Thread.run(Thread.java:829) ``` I belive this MAY have been caused by https://github.com/apache/iceberg/pull/7513/ which introduced a finalizer method for `org.apache.iceberg.aws.s3.S3FileIO`. I'm not entirely sure that this is the cause of this issue, but this finalizer seems problematic since it closes the client instance while not being the sole owner of it. `S3FileIO` may "leak" the client on these 3 methods ```java @Override public InputFile newInputFile(String path) { return S3InputFile.fromLocation(path, client(), s3FileIOProperties, metrics); } @Override public InputFile newInputFile(String path, long length) { return S3InputFile.fromLocation(path, length, client(), s3FileIOProperties, metrics); } @Override public OutputFile newOutputFile(String path) { return S3OutputFile.fromLocation(path, client(), s3FileIOProperties, metrics); } ``` If the caller retains ownership of a returned object (InputFIle, OutputFile, etc) this object may outlive the S3FileIO instance. The finalizer may be called and s3 client will be closed by the time it's needed. Again, I'm not sure if that's the case here since these things are super tricky to track down and I haven't managed to reproduce this locally. -- 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]
