apache / iceberg

Apache Iceberg
https://iceberg.apache.org/
Apache License 2.0
6.33k stars 2.19k forks source link

java.lang.IllegalStateException: Connection pool shut down when refreshing table metadata on s3 #8601

Closed igorcalabria closed 1 year ago

igorcalabria commented 1 year ago

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

  @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.

Kontinuation commented 1 year ago

I have encountered the same issue on Apache Iceberg 1.3.1. This exception is raised when reading parquet data file:

23/09/21 11:23:15 WARN TaskSetManager: Lost task 440.0 in stage 53.0 (TID 43464) (10.0.154.227 executor 8): java.lang.IllegalStateException: Connection pool shut down
    at software.amazon.awssdk.thirdparty.org.apache.http.util.Asserts.check(Asserts.java:34)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.requestConnection(PoolingHttpClientConnectionManager.java:269)
    at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$DelegatingHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:75)
    at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$InstrumentedHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:57)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:176)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
    at software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
    at software.amazon.awssdk.http.apache.internal.impl.ApacheSdkHttpClient.execute(ApacheSdkHttpClient.java:72)
    at software.amazon.awssdk.http.apache.ApacheHttpClient.execute(ApacheHttpClient.java:254)
    at software.amazon.awssdk.http.apache.ApacheHttpClient.access$500(ApacheHttpClient.java:104)
    at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:231)
    at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:228)
    at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:63)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:77)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:56)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:39)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:73)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
    at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
    at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56)
    at software.amazon.awssdk.services.sts.DefaultStsClient.assumeRoleWithWebIdentity(DefaultStsClient.java:755)
    at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.getUpdatedCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:73)
    at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.updateSessionCredentials(StsCredentialsProvider.java:88)
    at software.amazon.awssdk.utils.cache.CachedSupplier.lambda$jitteredPrefetchValueSupplier$3(CachedSupplier.java:284)
    at software.amazon.awssdk.utils.cache.CachedSupplier$PrefetchStrategy.fetch(CachedSupplier.java:420)
    at software.amazon.awssdk.utils.cache.CachedSupplier.refreshCache(CachedSupplier.java:199)
    at software.amazon.awssdk.utils.cache.CachedSupplier.get(CachedSupplier.java:128)
    at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.resolveCredentials(StsCredentialsProvider.java:99)
    at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.resolveCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:44)
    at software.amazon.awssdk.services.sts.internal.StsWebIdentityCredentialsProviderFactory$StsWebIdentityCredentialsProvider.resolveCredentials(StsWebIdentityCredentialsProviderFactory.java:93)
    at software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider.resolveCredentials(WebIdentityTokenFileCredentialsProvider.java:113)
    at software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.resolveCredentials(AwsCredentialsProviderChain.java:90)
    at software.amazon.awssdk.auth.credentials.internal.LazyAwsCredentialsProvider.resolveCredentials(LazyAwsCredentialsProvider.java:45)
    at software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.resolveCredentials(DefaultCredentialsProvider.java:126)
    at software.amazon.awssdk.core.internal.util.MetricUtils.measureDuration(MetricUtils.java:50)
    at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.resolveCredentials(AwsCredentialsAuthorizationStrategy.java:100)
    at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.addCredentialsToExecutionAttributes(AwsCredentialsAuthorizationStrategy.java:77)
    at software.amazon.awssdk.awscore.internal.AwsExecutionContextBuilder.invokeInterceptorsAndCreateExecutionContext(AwsExecutionContextBuilder.java:123)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.invokeInterceptorsAndCreateExecutionContext(AwsSyncClientHandler.java:69)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:64)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:62)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:63)
    at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:4478)
    at org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:192)
    at org.apache.iceberg.aws.s3.S3InputStream.positionStream(S3InputStream.java:177)
    at org.apache.iceberg.aws.s3.S3InputStream.read(S3InputStream.java:107)
    at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:102)
    at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFullyHeapBuffer(DelegatingSeekableInputStream.java:127)
    at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:91)
    at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader$ConsecutivePartList.readAll(ParquetFileReader.java:1850)
    at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader.internalReadRowGroup(ParquetFileReader.java:990)
    at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:940)
    at org.apache.iceberg.parquet.ParquetReader$FileIterator.advance(ParquetReader.java:147)
    at org.apache.iceberg.parquet.ParquetReader$FileIterator.next(ParquetReader.java:126)
    at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:65)
    at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:49)
    at org.apache.iceberg.spark.source.BaseReader.next(BaseReader.java:135)
    at org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:120)
    at org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:158)
    at org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1(DataSourceRDD.scala:63)
    at org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(DataSourceRDD.scala:63)
    at scala.Option.exists(Option.scala:376)
    at org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
    at org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.advanceToNextIter(DataSourceRDD.scala:97)
    at org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
    at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
    at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
    at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:513)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.hashAgg_doAggregateWithoutKey_0$(Unknown Source)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
    at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
    at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
    at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
    at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:101)
    at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
    at org.apache.spark.scheduler.Task.run(Task.scala:139)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
    at java.base/java.lang.Thread.run(Unknown Source)
nastra commented 1 year ago

Between Iceberg 1.2.1 and 1.3.0 the underlying http client changed from url-connection-client to the apache http client (#7119). It's possible that this might cause the issue you're seeing. You could try and switch back to using the url-connection-client in order to see if that fixes the issue.

Kontinuation commented 1 year ago

I think I have spotted the problem. This problem only reproduces when using web identity token file to authenticate. The WebIdentityTokenFileCredentialsProvider uses an STS client internally to refresh the credential, and the STS client was closed by the finalizer of S3FileIO. Here is the stacktrace of calling the shutdown() method of the pooling connection manager used by the STS client:

   java.base/java.lang.Thread.getStackTrace(Unknown Source)
   software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.printStackTrace(PoolingHttpClientConnectionManager.java:553)
   software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.shutdown(PoolingHttpClientConnectionManager.java:422)
   software.amazon.awssdk.http.apache.ApacheHttpClient.close(ApacheHttpClient.java:247)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.utils.AttributeMap.lambda$close$0(AttributeMap.java:87)
   java.base/java.util.HashMap$Values.forEach(Unknown Source)
   software.amazon.awssdk.utils.AttributeMap.close(AttributeMap.java:87)
   software.amazon.awssdk.core.client.config.SdkClientConfiguration.close(SdkClientConfiguration.java:79)
   software.amazon.awssdk.core.internal.http.HttpClientDependencies.close(HttpClientDependencies.java:80)
   software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient.close(AmazonSyncHttpClient.java:73)
   software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.close(BaseSyncClientHandler.java:88)
   software.amazon.awssdk.services.sts.DefaultStsClient.close(DefaultStsClient.java:1344)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.services.sts.internal.StsWebIdentityCredentialsProviderFactory$StsWebIdentityCredentialsProvider.close(StsWebIdentityCredentialsProviderFactory.java:99)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider.close(WebIdentityTokenFileCredentialsProvider.java:132)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.lambda$close$2(AwsCredentialsProviderChain.java:122)
   java.base/java.util.ArrayList.forEach(Unknown Source)
   java.base/java.util.Collections$UnmodifiableCollection.forEach(Unknown Source)
   software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.close(AwsCredentialsProviderChain.java:122)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.utils.Lazy.close(Lazy.java:77)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.auth.credentials.internal.LazyAwsCredentialsProvider.close(LazyAwsCredentialsProvider.java:50)
   software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.close(DefaultCredentialsProvider.java:131)
   software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70)
   software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87)
   software.amazon.awssdk.utils.AttributeMap.lambda$close$0(AttributeMap.java:87)
   java.base/java.util.HashMap$Values.forEach(Unknown Source)
   software.amazon.awssdk.utils.AttributeMap.close(AttributeMap.java:87)
   software.amazon.awssdk.core.client.config.SdkClientConfiguration.close(SdkClientConfiguration.java:79)
   software.amazon.awssdk.core.internal.http.HttpClientDependencies.close(HttpClientDependencies.java:80)
   software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient.close(AmazonSyncHttpClient.java:73)
   software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.close(BaseSyncClientHandler.java:88)
   software.amazon.awssdk.services.s3.DefaultS3Client.close(DefaultS3Client.java:11088)
   org.apache.iceberg.aws.s3.S3FileIO.close(S3FileIO.java:405)
   org.apache.iceberg.aws.s3.S3FileIO.finalize(S3FileIO.java:415)
   java.base/java.lang.System$2.invokeFinalize(Unknown Source)
   java.base/java.lang.ref.Finalizer.runFinalizer(Unknown Source)
   java.base/java.lang.ref.Finalizer$FinalizerThread.run(Unknown Source)

S3FileIO.close closes the Apache HTTP clients used by both the S3 client and the STS client (which is used by the web identity token file credential provider). The problem is that all S3FileIO objects actually shares one same credential provider. If any of the S3FileIO object was finalized, the shared credential provider object was broken.

AwsProperties.java line:1801 creates the credential provider object used by S3FileIO object, which is actually a singleton:

public final class DefaultCredentialsProvider {
  private static final DefaultCredentialsProvider DEFAULT_CREDENTIALS_PROVIDER = new DefaultCredentialsProvider(builder());
  // ...
  public static DefaultCredentialsProvider create() {
    return DEFAULT_CREDENTIALS_PROVIDER;
  }
  // ...
}

A workaround for this problem is always creating a new instance of DefaultCredentialsProvider object instead of using the singleton:

diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
index 9266c83f1..0f182a20c 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
@@ -45,6 +45,7 @@ import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
 import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.Builder;
 import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
 import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
 import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
@@ -1798,7 +1799,8 @@ public class AwsProperties implements Serializable {
       return credentialsProvider(this.clientCredentialsProvider);
     }

-    return DefaultCredentialsProvider.create();
+    Builder builder = DefaultCredentialsProvider.builder();
+    return builder.build();
   }

   private AwsCredentialsProvider credentialsProvider(String credentialsProviderClass) {

Currently, I'm still testing this patch to see if it actually resolve this issue. I'll let you know if the problem goes away.

Kontinuation commented 1 year ago

The patch mentioned above should have fixed this issue. However, this is quite hackish and coupled with the internal implementation of AWS SDK v2, which makes it hard to write unit tests for this patch. Is there any better approach for resolving this issue (except for switching to url-connection-client)?

nastra commented 1 year ago

@Kontinuation thanks for the detailed explanation. Can you confirm that this fixes the issue? I think changing to using DefaultCredentialsProvider.builder().build() makes sense. @amogh-jahagirdar thoughts on the issue described here?

igorcalabria commented 1 year ago

Nice catch. If I understood things correctly, the core issue is that AWS' SDK is tying the lifecycle of its clients to the credentials providers. If iceberg want's clients to be closeable (which is perfectly reasonable ), I think it will need to create a new provider for each client. The main drawback is that it will have to refresh credentials for each new client...

If AWS supports it, a different http client instance could be passed to the credential provider so it wouldn't be closed with s3's

EDIT

When a custom client is explicitly provided to the builder

SdkHttpClient apacheHttpClient = ApacheHttpClient.create();

// Singletons: Use the s3Client and dynamoDbClient for all requests.
S3Client s3Client = 
    S3Client.builder()
            .httpClient(apacheHttpClient).build();

It's not closed automatically

When an ApacheHttpClient instance is shared, you must close it when it is ready to be disposed. The SDK will not close the instance when the service client is closed.

https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/http-configuration-apache.html

But this makes things a bit more complex

Kontinuation commented 1 year ago

Can you confirm that this fixes the issue? I think changing to using DefaultCredentialsProvider.builder().build() makes sense. @amogh-jahagirdar thoughts on the issue described here?

Yes. I confirmed that this really fixed the issue.

Using a unique instance of DefaultCredentialsProvider per S3 client is also the default behavior of AWS SDK v2 (See AwsDefaultClientBuilder.resolveCredentials). I think using DefaultCredentialsProvider.builder().build() is a reasonable fix, though it will send more requests for refresh credentials.

stevenzwu commented 1 year ago

@Kontinuation would you be interested in creating a PR?

I also think DefaultCredentialsProvider.builder().build() makes sense. DefaultCredentialsProvider.create() uses a static global singleton which doesn't pair well with object lifecycle management.

@elkhand also observed the same problem with Flink session cluster mode, where static global singleton tends to be problematic with object lifecycle management.

Kontinuation commented 1 year ago

@Kontinuation would you be interested in creating a PR?

Sure. I've submitted a pull request https://github.com/apache/iceberg/pull/8677

AkshayWise commented 1 year ago

@Kontinuation @stevenzwu I believe this fix was released over 1.4.0 over last week, but I am still getting this error over Flink (1.15) Iceberg jobs:

java.lang.IllegalStateException: Connection pool shut down
    at org.apache.http.util.Asserts.check(Asserts.java:34)
    at org.apache.http.impl.conn.PoolingHttpClientConnectionManager.requestConnection(PoolingHttpClientConnectionManager.java:269)
    at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$DelegatingHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:75)
    at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$InstrumentedHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:57)
    at org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:176)
    at org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)
    at org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
    at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
    at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
    at software.amazon.awssdk.http.apache.internal.impl.ApacheSdkHttpClient.execute(ApacheSdkHttpClient.java:72)
    at software.amazon.awssdk.http.apache.ApacheHttpClient.execute(ApacheHttpClient.java:254)
    at software.amazon.awssdk.http.apache.ApacheHttpClient.access$500(ApacheHttpClient.java:104)
    at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:231)
    at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:228)
    at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:63)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:77)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:56)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:39)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:73)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
    at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
    at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56)
    at software.amazon.awssdk.services.sts.DefaultStsClient.assumeRoleWithWebIdentity(DefaultStsClient.java:755)
    at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.getUpdatedCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:73)
    at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.updateSessionCredentials(StsCredentialsProvider.java:88)
    at software.amazon.awssdk.utils.cache.CachedSupplier.lambda$jitteredPrefetchValueSupplier$3(CachedSupplier.java:284)
    at software.amazon.awssdk.utils.cache.CachedSupplier$PrefetchStrategy.fetch(CachedSupplier.java:420)
    at software.amazon.awssdk.utils.cache.CachedSupplier.refreshCache(CachedSupplier.java:199)
    at software.amazon.awssdk.utils.cache.CachedSupplier.get(CachedSupplier.java:128)
    at software.amazon.awssdk.services.sts.auth.StsCredentialsProvider.resolveCredentials(StsCredentialsProvider.java:99)
    at software.amazon.awssdk.services.sts.auth.StsAssumeRoleWithWebIdentityCredentialsProvider.resolveCredentials(StsAssumeRoleWithWebIdentityCredentialsProvider.java:44)
    at software.amazon.awssdk.services.sts.internal.StsWebIdentityCredentialsProviderFactory$StsWebIdentityCredentialsProvider.resolveCredentials(StsWebIdentityCredentialsProviderFactory.java:93)
    at software.amazon.awssdk.auth.credentials.WebIdentityTokenFileCredentialsProvider.resolveCredentials(WebIdentityTokenFileCredentialsProvider.java:113)
    at software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.resolveCredentials(AwsCredentialsProviderChain.java:90)
    at software.amazon.awssdk.auth.credentials.internal.LazyAwsCredentialsProvider.resolveCredentials(LazyAwsCredentialsProvider.java:45)
    at software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.resolveCredentials(DefaultCredentialsProvider.java:126)
    at software.amazon.awssdk.core.internal.util.MetricUtils.measureDuration(MetricUtils.java:50)
    at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.resolveCredentials(AwsCredentialsAuthorizationStrategy.java:100)
    at software.amazon.awssdk.awscore.internal.authcontext.AwsCredentialsAuthorizationStrategy.addCredentialsToExecutionAttributes(AwsCredentialsAuthorizationStrategy.java:77)
    at software.amazon.awssdk.awscore.internal.AwsExecutionContextBuilder.invokeInterceptorsAndCreateExecutionContext(AwsExecutionContextBuilder.java:123)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.invokeInterceptorsAndCreateExecutionContext(AwsSyncClientHandler.java:69)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:64)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:62)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:63)
    at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:4478)
    at org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:192)
    at org.apache.iceberg.aws.s3.S3InputStream.positionStream(S3InputStream.java:177)
    at org.apache.iceberg.aws.s3.S3InputStream.read(S3InputStream.java:107)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1744)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1143)
    at org.apache.iceberg.shaded.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3809)
    at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:273)
    at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:266)
    at org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$0(BaseMetastoreTableOperations.java:189)
    at org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$1(BaseMetastoreTableOperations.java:208)
    at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
    at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
    at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
    at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
    at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:208)
    at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:185)
    at org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:176)
    at org.apache.iceberg.aws.glue.GlueTableOperations.doRefresh(GlueTableOperations.java:141)
    at org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:97)
    at org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:80)
    at org.apache.iceberg.BaseMetastoreCatalog.loadTable(BaseMetastoreCatalog.java:47)
    at org.apache.iceberg.flink.TableLoader$CatalogTableLoader.loadTable(TableLoader.java:133)
    at org.apache.iceberg.flink.sink.IcebergFilesCommitter.initializeState(IcebergFilesCommitter.java:151)
    at org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:122)
    at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:283)
    at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:106)
    at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:703)
    at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
    at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:679)
    at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:646)
    at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
    at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:917)
    at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
    at java.base/java.lang.Thread.run(Unknown Source)
nastra commented 1 year ago

@AkshayWise this fix didn't make it into 1.4.0 unfortunately

elkhand commented 11 months ago

This issue still exists in the Iceberg 1.14.1 version with Flink 1.17 when the iceberg catalog is created with 'io-impl'='org.apache.iceberg.aws.s3.S3FileIO'. The issue happens only on the Iceberg read path, but not in the write path, and happens only in FlinkSQL, but not in PyFlink.

If the iceberg catalog is created without S3FileIO, then this issue does not occur:

CREATE CATALOG iceberg with(
   'type'='iceberg',
   'catalog-type'='hive',
   'uri'='thrift://<hms_endpoint>:9083',
   'clients'='1',
   'property-version'='1',
   'warehouse'='s3a://<s3_path>/hive/warehouse'
   -- 'io-impl'='org.apache.iceberg.aws.s3.S3FileIO'
);
nastra commented 11 months ago

@elkhand can you paste the stacktrace that you're seeing? This would help in seeing whether it's the same or a different issue.

elkhand commented 11 months ago

Sure @nastra , here is the stacktrace:

{"exception":{
"exception_class":"org.apache.flink.util.FlinkException","exception_message":"Global failure triggered by OperatorCoordinator for 'Source: xyz_iceberg_table[1]' 
(operator bc764cd8ddf7a0cff126f51c16239658).",
"stacktrace":"org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'Source: xyz_iceberg_table[1]' 
(operator bc764cd8ddf7a0cff126f51c16239658).\n\t
at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:624)\n\t
at org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$QuiesceableContext.failJob(RecreateOnResetOperatorCoordinator.java:243)\n\t
at org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext.failJob(SourceCoordinatorContext.java:395)\n\tat org.apache.flink.runtime.source.coordinator.SourceCoordinator.start(SourceCoordinator.java:221)\n\t
at org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$DeferrableCoordinator.resetAndStart(RecreateOnResetOperatorCoordinator.java:411)\n\t
at org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator.lambda$resetToCheckpoint$6(RecreateOnResetOperatorCoordinator.java:151)\n\tat java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)\n\t
at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)\n\t
at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)\n\t
at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2079)\n\t
at org.apache.flink.runtime.operators.coordination.ComponentClosingUtils.lambda$closeAsyncWithTimeout$0(ComponentClosingUtils.java:77)\n\tat java.base/java.lang.Thread.run(Thread.java:829)\n

Caused by: java.lang.IllegalStateException: Connection pool shut down\n\t
at software.amazon.awssdk.thirdparty.org.apache.http.util.Asserts.check(Asserts.java:34)\n\t
at software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.requestConnection(PoolingHttpClientConnectionManager.java:269)\n\t
at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$DelegatingHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:75)\n\t
at software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$InstrumentedHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:57)\n\t
at software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:176)\n\t
at software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)\n\tat software.amazon.awssdk.thirdparty.org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)\n\t
at software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)\n\tat software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)\n\t
at software.amazon.awssdk.http.apache.internal.impl.ApacheSdkHttpClient.execute(ApacheSdkHttpClient.java:72)\n\t
at software.amazon.awssdk.http.apache.ApacheHttpClient.execute(ApacheHttpClient.java:254)\n\t
at software.amazon.awssdk.http.apache.ApacheHttpClient.access$500(ApacheHttpClient.java:104)\n\t
at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:231)\n\t
at software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:228)\n\t
at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:63)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:77)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:56)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:39)\n\tat software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)\n\t
at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)\n\t
at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)\n\t
at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193)\n\t
at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)\n\t
at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171)\n\t
at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82)\n\t
at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)\n\t
at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76)\n\t
at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)\n\t
at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56)\n\t
at software.amazon.awssdk.services.s3.DefaultS3Client.headObject(DefaultS3Client.java:5541)\n\t
at org.apache.iceberg.aws.s3.BaseS3File.getObjectMetadata(BaseS3File.java:85)\n\t
at org.apache.iceberg.aws.s3.S3InputFile.getLength(S3InputFile.java:75)\n\t
at org.apache.iceberg.avro.AvroIterable.newFileReader(AvroIterable.java:100)\n\t
at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:76)\n\t
at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:36)\n\t
at org.apache.iceberg.relocated.com.google.common.collect.Iterables.addAll(Iterables.java:337)\n\t
at org.apache.iceberg.relocated.com.google.common.collect.Lists.newLinkedList(Lists.java:241)\n\t
at org.apache.iceberg.ManifestLists.read(ManifestLists.java:45)\n\t
at org.apache.iceberg.BaseSnapshot.cacheManifests(BaseSnapshot.java:148)\n\t
at org.apache.iceberg.BaseSnapshot.dataManifests(BaseSnapshot.java:174)\n\t
at org.apache.iceberg.DataTableScan.doPlanFiles(DataTableScan.java:73)\n\t
at org.apache.iceberg.SnapshotScan.planFiles(SnapshotScan.java:132)\n\t
at org.apache.iceberg.BaseTableScan.planTasks(BaseTableScan.java:44)\n\t
at org.apache.iceberg.DataTableScan.planTasks(DataTableScan.java:27)\n\t
at org.apache.iceberg.flink.source.FlinkSplitPlanner.planTasks(FlinkSplitPlanner.java:136)\n\t
at org.apache.iceberg.flink.source.FlinkSplitPlanner.planIcebergSourceSplits(FlinkSplitPlanner.java:76)\n\t
at org.apache.iceberg.flink.source.IcebergSource.planSplitsForBatch(IcebergSource.java:116)\n\t
at org.apache.iceberg.flink.source.IcebergSource.createEnumerator(IcebergSource.java:194)\n\t
at org.apache.iceberg.flink.source.IcebergSource.createEnumerator(IcebergSource.java:155)\n\t
at org.apache.flink.runtime.source.coordinator.SourceCoordinator.start(SourceCoordinator.java:217)\n\t
... 8 more\n"},
"@version":1,"source_host":"single-sql-script-deployment-7b9b588c5f-9v4qf",
"message":"Trying to recover from a global failure.","thread_name":"flink-akka.actor.default-dispatcher-16","
@timestamp":"2023-11-15T10:44:43.358+00:00","level":"INFO","logger_name":"org.apache.flink.runtime.jobmaster.JobMaster"}
Kontinuation commented 11 months ago

It does not seem to be the same problem. According to the stacktrace, Connection pool shut down was not raised by an STS client. There could be some other problems with the lifecycle management of HTTP clients.

mas-chen commented 11 months ago

It does not seem to be the same problem. According to the stacktrace, Connection pool shut down was not raised by an STS client. There could be some other problems with the lifecycle management of HTTP clients.

Yup, this is not the same issue which you can tell via the new stacktrace. I'm working on the fix for this new issue but it is pending some internal company processes. I'll likely submit a PR after Thanksgiving week.

shanzi commented 9 months ago

@mas-chen Hi, I'm wondering is there any updates or work around for the connection pool shut down issue? I'm also faced with the same problem.

javrasya commented 9 months ago

I had the same issue you had @elkhand and @Kontinuation is right. Something is closing the S3 client and I don't know what that is. I ended up writing a custom S3FileIO and use it with my catalog loader which re-opens the s3 client if it is ever closed. That way I don't have the problem.

Note: It does not work with Iceberg version 1.3 because the methods that this custom S3FileIO overriding is only public since Iceberg version 1.4.

https://gist.github.com/javrasya/513f838a8af355b51506ca2a2dc1e3d8

mas-chen commented 9 months ago

The issue that @elkhand mentioned is resolved by https://github.com/apache/iceberg/pull/9173. I would expect this to be included in the next Iceberg release and supported by 1.16, 1.17, and 1.18 Flink versions.