davseitsev opened a new issue, #11633:
URL: https://github.com/apache/iceberg/issues/11633

   ### Apache Iceberg version
   
   1.7.0 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   We have a maintenance job which run all necessary SparkActions on our data 
lake. Recently we have upgraded to iceberg 1.7.0 and switched to 
`org.apache.iceberg.aws.s3.S3FileIO`. After this we started to get 
`java.lang.IllegalStateException: Connection pool shut` errors on both driver 
and workers.
   
   Here is example stacktrace:
   ```
   java.lang.IllegalStateException: Connection pool shut down
        at 
software.amazon.awssdk.thirdparty.org.apache.http.util.Asserts.check(Asserts.java:34)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.requestConnection(PoolingHttpClientConnectionManager.java:269)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$DelegatingHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:75)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.internal.conn.ClientConnectionManagerFactory$InstrumentedHttpClientConnectionManager.requestConnection(ClientConnectionManagerFactory.java:57)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:176)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:186)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:185)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:83)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:56)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.internal.impl.ApacheSdkHttpClient.execute(ApacheSdkHttpClient.java:72)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient.execute(ApacheHttpClient.java:254)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient.access$500(ApacheHttpClient.java:104)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:231)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient$1.call(ApacheHttpClient.java:228)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:99)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:79)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:57)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:40)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:74)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:43)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:79)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:41)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:55)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.executeRequest(RetryableStage2.java:93)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.java:56)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.java:36)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:53)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:35)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:82)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:62)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:43)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:210)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:66)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:60)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:60)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:5441)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.DelegatingS3Client.lambda$getObject$43(DelegatingS3Client.java:4386)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.internal.crossregion.S3CrossRegionSyncClient.invokeOperation(S3CrossRegionSyncClient.java:67)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.DelegatingS3Client.getObject(DelegatingS3Client.java:4386)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:240) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:225) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.aws.s3.S3InputStream.positionStream(S3InputStream.java:221) 
~[iceberg-spark3-runtime.jar:?]
        at org.apache.iceberg.aws.s3.S3InputStream.read(S3InputStream.java:143) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1744)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1143)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.shaded.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3809)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:280) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:273) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$0(BaseMetastoreTableOperations.java:179)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$1(BaseMetastoreTableOperations.java:198)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219) 
~[iceberg-spark3-runtime.jar:?]
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203) 
~[iceberg-spark3-runtime.jar:?]
        at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:198)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:175)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:170)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.hive.HiveTableOperations.doRefresh(HiveTableOperations.java:167)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:87)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:70)
 ~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.BaseMetastoreCatalog.loadTable(BaseMetastoreCatalog.java:50) 
~[iceberg-spark3-runtime.jar:?]
        at org.apache.iceberg.spark.SparkCatalog.load(SparkCatalog.java:845) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:170) 
~[iceberg-spark3-runtime.jar:?]
        at 
org.apache.spark.sql.connector.catalog.CatalogV2Util$.getTable(CatalogV2Util.scala:355)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.connector.catalog.CatalogV2Util$.loadTable(CatalogV2Util.scala:336)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.$anonfun$resolveRelation$5(Analyzer.scala:1425)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at scala.Option.orElse(Option.scala:447) ~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.$anonfun$resolveRelation$1(Analyzer.scala:1424)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at scala.Option.orElse(Option.scala:447) ~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$resolveRelation(Analyzer.scala:1409)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$14.applyOrElse(Analyzer.scala:1222)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$14.applyOrElse(Analyzer.scala:1201)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:167)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:77) 
~[spark-sql-api_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:167)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:353)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:163)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:159)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:34)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:1201)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:1160)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:239)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) 
~[scala-library-2.12.18.jar:?]
        at 
scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) 
~[scala-library-2.12.18.jar:?]
        at scala.collection.immutable.List.foldLeft(List.scala:91) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.executeBatch$1(RuleExecutor.scala:236)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$6(RuleExecutor.scala:319)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$RuleExecutionContext$.withContext(RuleExecutor.scala:368)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$5(RuleExecutor.scala:319)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$5$adapted(RuleExecutor.scala:309)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at scala.collection.immutable.List.foreach(List.scala:431) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:309)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:195)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:191)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeSameContext(Analyzer.scala:303)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$2(Analyzer.scala:299)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:216)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:299) 
~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:245) 
~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:270)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:360)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:269)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:93)
 ~[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:219)
 ~[spark-catalyst_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:277)
 ~[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:711)
 ~[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:277)
 ~[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901) 
[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:276)
 [spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:93)
 [spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:90) 
[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:101)
 [spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:99)
 [spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:164)
 [spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.DataFrameWriterV2.runCommand(DataFrameWriterV2.scala:198) 
[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.sql.DataFrameWriterV2.append(DataFrameWriterV2.scala:150) 
[spark-sql_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
       ...
   ```
   
   I managed to track where `PoolingHttpClientConnectionManager.shutdown()` 
method is called from.
   
   1. First place is `FileIOTracker`: 
https://github.com/apache/iceberg/blob/cf02ffac4329141b30bca265cafb9987f64f6cc4/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java#L41-L46
   Here is a stack trace:
   ```
   java.lang.Exception: shutting down from
           at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.shutdown(PoolingHttpClientConnectionManager.java:434)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.http.apache.ApacheHttpClient.close(ApacheHttpClient.java:247)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.utils.AttributeMap.closeIfPossible(AttributeMap.java:678)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.utils.AttributeMap.access$1600(AttributeMap.java:49) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.utils.AttributeMap$DerivedValue.close(AttributeMap.java:632)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at java.util.HashMap$Values.forEach(HashMap.java:1065) ~[?:?]
           at 
software.amazon.awssdk.utils.AttributeMap.close(AttributeMap.java:107) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.core.client.config.SdkClientConfiguration.close(SdkClientConfiguration.java:118)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.core.internal.http.HttpClientDependencies.close(HttpClientDependencies.java:82)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient.close(AmazonSyncHttpClient.java:76)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.close(BaseSyncClientHandler.java:86)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.services.s3.DefaultS3Client.close(DefaultS3Client.java:12516)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at 
software.amazon.awssdk.services.s3.DelegatingS3Client.close(DelegatingS3Client.java:10140)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
           at org.apache.iceberg.aws.s3.S3FileIO.close(S3FileIO.java:416) ~[?:?]
           at 
org.apache.iceberg.io.FileIOTracker.lambda$new$0(FileIOTracker.java:44) ~[?:?]
           at 
org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$notifyRemoval$1(BoundedLocalCache.java:327)
 ~[?:?]
           at 
java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1395)
 [?:?]
           at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373) 
[?:?]
           at 
java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
 [?:?]
           at java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655) 
[?:?]
           at 
java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622) [?:?]
           at 
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165) 
[?:?]
   ```
   We use `HiveCatalog`, it creates FileIO one time in the `initialize` method:
   
https://github.com/apache/iceberg/blob/e449d3405cfdb304c94835845bd8f34a73b4a517/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L122-L125
   And each time it creates new `TableOperations`, it puts a new item to the 
`FileIOTracker` but with the same instance of `FileIO`
   
https://github.com/apache/iceberg/blob/e449d3405cfdb304c94835845bd8f34a73b4a517/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L629-L636
   When some `TableOperations` is cleaned up by the GC, `FileIO` is closed and 
everything starts failing with `Connection pool shut down` error.
   This is what we got on Spark Driver. I patched `FileIOTracker` not to touch 
`FileIO` and the driver is fine now.
   
   2. On Spark workers we have the same problem. Here are stack traces where 
`PoolingHttpClientConnectionManager` is close from:
   ```
   java.lang.Exception: shutting down from
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.shutdown(PoolingHttpClientConnectionManager.java:412)
 [__app__.jar:?]
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.finalize(PoolingHttpClientConnectionManager.java:207)
 [__app__.jar:?]
        at java.lang.System$2.invokeFinalize(System.java:2314) [?:?]
        at java.lang.ref.Finalizer.runFinalizer(Finalizer.java:88) [?:?]
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:173) [?:?]
   ```
   This one looks like GC on unused `PoolingHttpClientConnectionManager` and 
should not cause the problem.
   
   
   ```
   java.lang.Exception: shutting down from
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.shutdown(PoolingHttpClientConnectionManager.java:412)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient.close(ApacheHttpClient.java:247)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap.closeIfPossible(AttributeMap.java:678)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap.access$1600(AttributeMap.java:49) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap$DerivedValue.close(AttributeMap.java:632)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at java.util.HashMap$Values.forEach(HashMap.java:1065) [?:?]
        at 
software.amazon.awssdk.utils.AttributeMap.close(AttributeMap.java:107) 
[__app__.jar:?]
        at 
software.amazon.awssdk.core.client.config.SdkClientConfiguration.close(SdkClientConfiguration.java:118)
 [__app__.jar:?]
        at 
software.amazon.awssdk.core.internal.http.HttpClientDependencies.close(HttpClientDependencies.java:82)
 [__app__.jar:?]
        at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient.close(AmazonSyncHttpClient.java:76)
 [__app__.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.close(BaseSyncClientHandler.java:86)
 [__app__.jar:?]
        at 
software.amazon.awssdk.services.s3.DefaultS3Client.close(DefaultS3Client.java:12516)
 [__app__.jar:?]
        at 
software.amazon.awssdk.services.s3.DelegatingS3Client.close(DelegatingS3Client.java:10140)
 [__app__.jar:?]
        at org.apache.iceberg.aws.s3.S3FileIO.close(S3FileIO.java:416) 
[iceberg-spark3-runtime.jar:?]
        at org.apache.iceberg.aws.s3.S3FileIO.finalize(S3FileIO.java:426) 
[iceberg-spark3-runtime.jar:?]
        at java.lang.System$2.invokeFinalize(System.java:2314) [?:?]
        at java.lang.ref.Finalizer.runFinalizer(Finalizer.java:88) [?:?]
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:173) [?:?]
   ```
   This one looks like GC on unused `S3FileIO` and should not cause the problem 
as well.
   
   
   But the following one seems problematic:
   ```
   java.lang.Exception: shutting down from
        at 
software.amazon.awssdk.thirdparty.org.apache.http.impl.conn.PoolingHttpClientConnectionManager.shutdown(PoolingHttpClientConnectionManager.java:412)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.http.apache.ApacheHttpClient.close(ApacheHttpClient.java:247)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at software.amazon.awssdk.utils.IoUtils.closeQuietly(IoUtils.java:70) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.IoUtils.closeIfCloseable(IoUtils.java:87) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap.closeIfPossible(AttributeMap.java:678)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap.access$1600(AttributeMap.java:49) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.utils.AttributeMap$DerivedValue.close(AttributeMap.java:632)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at java.util.HashMap$Values.forEach(HashMap.java:1065) ~[?:?]
        at 
software.amazon.awssdk.utils.AttributeMap.close(AttributeMap.java:107) 
~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.client.config.SdkClientConfiguration.close(SdkClientConfiguration.java:118)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.HttpClientDependencies.close(HttpClientDependencies.java:82)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient.close(AmazonSyncHttpClient.java:76)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.close(BaseSyncClientHandler.java:86)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.DefaultS3Client.close(DefaultS3Client.java:12516)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at 
software.amazon.awssdk.services.s3.DelegatingS3Client.close(DelegatingS3Client.java:10140)
 ~[aws-sdk-java-bundle-2.28.8.jar:?]
        at org.apache.iceberg.aws.s3.S3FileIO.close(S3FileIO.java:416) ~[?:?]
        at 
org.apache.iceberg.spark.source.SerializableTableWithSize.close(SerializableTableWithSize.java:69)
 ~[?:?]
        at 
org.apache.spark.storage.memory.MemoryStore.$anonfun$freeMemoryEntry$1(MemoryStore.scala:410)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.memory.MemoryStore.$anonfun$freeMemoryEntry$1$adapted(MemoryStore.scala:407)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) 
~[scala-library-2.12.18.jar:?]
        at 
scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) 
~[scala-library-2.12.18.jar:?]
        at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.storage.memory.MemoryStore.freeMemoryEntry(MemoryStore.scala:407)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.memory.MemoryStore.remove(MemoryStore.scala:425) 
~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.BlockManager.removeBlockInternal(BlockManager.scala:2126)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.BlockManager.removeBlock(BlockManager.scala:2108) 
~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.BlockManager.$anonfun$removeBroadcast$3(BlockManager.scala:2079)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.BlockManager.$anonfun$removeBroadcast$3$adapted(BlockManager.scala:2079)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at scala.collection.Iterator.foreach(Iterator.scala:943) 
~[scala-library-2.12.18.jar:?]
        at scala.collection.Iterator.foreach$(Iterator.scala:943) 
~[scala-library-2.12.18.jar:?]
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.storage.BlockManager.removeBroadcast(BlockManager.scala:2079) 
~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
org.apache.spark.storage.BlockManagerStorageEndpoint$$anonfun$receiveAndReply$1.$anonfun$applyOrElse$4(BlockManagerStorageEndpoint.scala:69)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at 
scala.runtime.java8.JFunction0$mcI$sp.apply(JFunction0$mcI$sp.java:23) 
~[scala-library-2.12.18.jar:?]
        at 
org.apache.spark.storage.BlockManagerStorageEndpoint.$anonfun$doAsync$1(BlockManagerStorageEndpoint.scala:101)
 ~[spark-core_2.12-3.5.2-amzn-1.jar:3.5.2-amzn-1]
        at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659) 
~[scala-library-2.12.18.jar:?]
        at scala.util.Success.$anonfun$map$1(Try.scala:255) 
~[scala-library-2.12.18.jar:?]
        at scala.util.Success.map(Try.scala:213) ~[scala-library-2.12.18.jar:?]
        at scala.concurrent.Future.$anonfun$map$1(Future.scala:292) 
~[scala-library-2.12.18.jar:?]
        at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33) 
~[scala-library-2.12.18.jar:?]
        at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33) 
~[scala-library-2.12.18.jar:?]
        at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) 
[scala-library-2.12.18.jar:?]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) 
[?:?]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) 
[?:?]
        at java.lang.Thread.run(Thread.java:840) [?:?]
   ```
   
   When `SerializableTableWithSize` is closed, it closes underlying 
`TableOperations` and associated `FileIO`. As we have single instance of 
`FileIO`, after this all the tasks on the worker start failing with `Connection 
pool shut down`. I'm not sure how to handle this case.
   
   ### Willingness to contribute
   
   - [ ] I can contribute a fix for this bug independently
   - [ ] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [X] I cannot contribute a fix for this bug at this time


-- 
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: issues-unsubscr...@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to