fpetersen-gl opened a new issue, #13508:
URL: https://github.com/apache/iceberg/issues/13508

   ### Apache Iceberg version
   
   1.9.1 (latest release)
   
   ### Query engine
   
   None
   
   ### Please describe the bug 🐞
   
   ## Description of the problem  
   If the connection to the underlying IO (we're using S3) is cut in the wrong 
moment, a `parquet`-file cannot be written correctly, but its writer assumes it 
was.
   This writer is in our case wrapped by a `FanoutWriter`, which happily 
returns this file as result after closing all of them, making it appear in the 
metadata.
   The next call to close all writers being part of the `FanoutWriter` ignores 
the failed writer, as it is already marked as closed.
   
   Result: A snapshot is being written which references the missing 
parquet-file.
   
   ## Code analysis
   Setting the internal state to `closed` in 
[`ParquetWriter.close()`](https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java#L255)
 happens already before any code is executed that could potentially break. 
There's 
[`writer.end(metadata)`](https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java#L261),
 which can throw an `IOException`, but 
[`flushRowGroup(true)`](https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java#L258)
 before can throw an `UncheckedIOException`, thus leaving the writer in a wrong 
state.
   
   ## Possible solution
   First naive thing that comes to my mind would be to move `this.closed = 
true` to the very end of the method. This would only change the state iff all 
procedures for closing the writer have been executed successfully.
   I'll try to come up with a test to reproduce this issue, will update the 
ticket afterwards.
   
   ### Willingness to contribute
   
   - [x] I can contribute a fix for this bug independently
   - [x] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [ ] 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