Parquet: avoid premature closing of FSDataInputStream object#3276
Parquet: avoid premature closing of FSDataInputStream object#3276velctor wants to merge 2 commits intoapache:mainfrom
Conversation
| public SeekableInputStream newStream() throws IOException { | ||
| return stream(file.newStream()); | ||
| icebergHadoopStream = file.newStream(); | ||
| return stream(icebergHadoopStream); |
There was a problem hiding this comment.
This doesn't look correct to me. The InputFile is a factory interface and is not responsible for keeping references to the streams that are created.
There was a problem hiding this comment.
I know that, but the org.apache.iceberg.io.SeekableStream object will be GC after this.f = file.newStream(); in org.apache.parquet.hadoop.ParquetFileReader class. It is difficult to modify the code in the parquet library, so this may be an easier way to solve this problem, under the premise of keeping #90. I want to know if you have any other ideas.
|
@velctor, can you post the problem that you're seeing? If you have a stream that was closed by the finalizer, how is anything still using it to hit the |
The error |
|
@velctor, why do you think that the stream was closed by the finalizer and not by a normal call to |
In order to determine that the stream closed by the From the In addition, when I applied the changes in this PR to iceberg, no related errors were reported after testing. |
|
@velctor, it looks like that is a different finalizer. If you were seeing something that was closed by the Iceberg finalizer, you'd get a stack trace for where the stream was created and it would be no longer referenced. I think in this case, your |
Oh, the stack information is too long, So I didn't add it before. The relevant complete information is as follows: I have added the call stack log of the |
|
Thanks for the logs! It's always helpful to have the full picture. I think what's happening is that Trino is extracting the underlying @electrum, can you help us take a look at this problem? Looks like something is getting the underlying stream from Iceberg and keeping a reference past when the reference to the Iceberg stream is garbage collected. |
|
We had this problem in Trino previously that @phd3 fixed in trinodb/trino#8504 @velctor It looks like you are using a patched version of the Trino Iceberg connector. Can you share the patch? I don't believe we see this issue today with the current version, so this might be related to the patch. Are you using a version of trinodb/trino#8534 from @jackye1995? |
|
Longer explanation of the problem: trinodb/trino#5201 (comment) |
|
Looking at this again with fresh eyes, the actual problem is that Trino's |
|
Thanks, @electrum! |
|
@rdblue @electrum Thanks very much! The version that I used is 359, I will merge with the PR trinodb/trino#8504 to test. |
|
This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions. |
|
This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time. |
#90 adds a finalizer that closes open streams when they are garbage collected. But the
org.apache.iceberg.io.SeekableStreamobject is discarded in theParquetIO#stream,andHadoopStream#finalizer()is called, which causes theFSDataOutputStreamto be closed early. This will result in deletefile reading fails withFailed to open Parquet file ...orStream closed.In order to avoid this, add the member variable
icebergHadoopStreamto theParquetIO$ParquetInputFileclass and and use it as the parameter of theParquetIO#stream.