zizon commented on PR #12079: URL: https://github.com/apache/iceberg/pull/12079#issuecomment-2612949803
I think I found the root cause. ``` 2025-01-24T10:39:58.963+0800 WARN Finalizer org.apache.iceberg.hadoop.HadoopStreams Unclosed input stream created by: org.apache.iceberg.hadoop.HadoopStreams$HadoopSeekableInputStream.<init>(HadoopStreams.java:91) org.apache.iceberg.hadoop.HadoopStreams.wrap(HadoopStreams.java:55) org.apache.iceberg.hadoop.HadoopInputFile.newStream(HadoopInputFile.java:183) com.facebook.presto.hive.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:23) com.facebook.presto.hive.HdfsEnvironment.doAs(HdfsEnvironment.java:80) com.facebook.presto.iceberg.HdfsInputFile.newStream(HdfsInputFile.java:58) org.apache.iceberg.parquet.ParquetIO$ParquetInputFile.newStream(ParquetIO.java:179). <-- this line org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:774) org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:658) org.apache.iceberg.parquet.ReadConf.newReader(ReadConf.java:231) org.apache.iceberg.parquet.ReadConf.<init>(ReadConf.java:80) org.apache.iceberg.parquet.ParquetReader.init(ParquetReader.java:71) org.apache.iceberg.parquet.ParquetReader.iterator(ParquetReader.java:91) org.apache.iceberg.io.CloseableIterable$7$1.<init>(CloseableIterable.java:188) org.apache.iceberg.io.CloseableIterable$7.iterator(CloseableIterable.java:187) org.apache.iceberg.io.CloseableIterable$ConcatCloseableIterable$ConcatCloseableIterator.hasNext(CloseableIterable.java:257) java.lang.Iterable.forEach(Iterable.java:74) org.apache.iceberg.deletes.Deletes.toPositionIndex(Deletes.java:138) ``` comments inline. [https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java#L96](https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java#L96) ```java static SeekableInputStream stream(org.apache.iceberg.io.SeekableInputStream stream) { if (stream instanceof DelegatingInputStream) { // for somehow, it tries to optimistic steal the underlying stream, creating a new one by wrapping it. // the caller close the wrapper and underlying stream but not this intermediate one. InputStream wrapped = ((DelegatingInputStream) stream).getDelegate(); if (wrapped instanceof FSDataInputStream) { return HadoopStreams.wrap((FSDataInputStream) wrapped); } } return new ParquetInputStreamAdapter(stream); } ``` A similar approach for the output version. Can we just go through the Adapter path? -- 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 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