hugofriant opened a new issue, #36438:
URL: https://github.com/apache/beam/issues/36438
### What happened?
## Description
Writing Iceberg files to S3 with AWS Glue Data Catalog fails with
`IllegalStateException: Connection pool shut down` error starting in Beam
v2.63. The pipeline works correctly in v2.62.
## Environment
- **Beam Version:** 2.63+
- **Runner:** Direc Runner or Flink Runner
- **Storage:** S3
- **Catalog:** AWS Glue Data Catalog
- **File Format:** Parquet
## Steps to Reproduce
1. Configure IcebergIO to write to S3 with AWS Glue Data Catalog
2. Use Beam 2.63 or later
3. Execute a pipeline that writes data using IcebergIO
## Stack Trace
```
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
org.apache.iceberg.aws.s3.S3OutputStream.completeUploads(S3OutputStream.java:444)
at
org.apache.iceberg.aws.s3.S3OutputStream.close(S3OutputStream.java:270)
at
org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:1204)
at
org.apache.iceberg.parquet.ParquetWriter.close(ParquetWriter.java:257)
at org.apache.iceberg.io.DataWriter.close(DataWriter.java:82)
at
org.apache.beam.sdk.io.iceberg.RecordWriter.close(RecordWriter.java:116)
at
org.apache.beam.sdk.io.iceberg.RecordWriterManager$DestinationState.lambda$new$0(RecordWriterManager.java:131)
Caused by: java.lang.IllegalArgumentException: Expected all data writers to
be closed, but found 1 data writer(s) still open
at
org.apache.beam.sdk.io.iceberg.RecordWriterManager.close(RecordWriterManager.java:363)
````
## Root Cause
I try to identify the root cause of the issue, but I think the issue is
related to the way the RecordWriter is closed in
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriter.java
```java
try (FileIO io = table.io()) {
OutputFile tmpFile = io.newOutputFile(absoluteFilename);
EncryptedOutputFile encryptedOutputFile =
table.encryption().encrypt(tmpFile);
outputFile = encryptedOutputFile.encryptingOutputFile();
keyMetadata = encryptedOutputFile.keyMetadata();
}
```
Then L130 the close method is failing
### Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
### Issue Components
- [ ] Component: Python SDK
- [x] Component: Java SDK
- [ ] Component: Go SDK
- [ ] Component: Typescript SDK
- [x] Component: IO connector
- [ ] Component: Beam YAML
- [ ] Component: Beam examples
- [ ] Component: Beam playground
- [ ] Component: Beam katas
- [ ] Component: Website
- [ ] Component: Infrastructure
- [ ] Component: Spark Runner
- [x] Component: Flink Runner
- [ ] Component: Samza Runner
- [ ] Component: Twister2 Runner
- [ ] Component: Hazelcast Jet Runner
- [ ] Component: Google Cloud Dataflow Runner
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]