gszadovszky commented on code in PR #3351:
URL: https://github.com/apache/parquet-java/pull/3351#discussion_r2459620460


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java:
##########
@@ -1812,6 +1813,8 @@ public void end(Map<String, String> extraMetaData) throws 
IOException {
       LOG.debug("{}: end", out.getPos());
       this.footer = new ParquetMetadata(new FileMetaData(schema, 
extraMetaData, Version.FULL_VERSION), blocks);
       serializeFooter(footer, out, fileEncryptor, metadataConverter);
+    } catch (Exception e) {
+      aborted = true;

Review Comment:
   We do not want to swallow the exception, just set the flag and re-throw.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java:
##########
@@ -140,8 +140,9 @@ public void close() throws IOException, 
InterruptedException {
         }
         finalMetadata.putAll(finalWriteContext.getExtraMetaData());
         parquetFileWriter.end(finalMetadata);
+        AutoCloseables.uncheckedClose(parquetFileWriter);
       } finally {
-        AutoCloseables.uncheckedClose(columnStore, pageStore, 
bloomFilterWriteStore, parquetFileWriter);
+        AutoCloseables.uncheckedClose(columnStore, pageStore, 
bloomFilterWriteStore);

Review Comment:
   Now that we have `ParquetFileWriter` to handle the "aborted" state, this 
change can be reverted.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java:
##########
@@ -1812,6 +1813,8 @@ public void end(Map<String, String> extraMetaData) throws 
IOException {
       LOG.debug("{}: end", out.getPos());
       this.footer = new ParquetMetadata(new FileMetaData(schema, 
extraMetaData, Version.FULL_VERSION), blocks);
       serializeFooter(footer, out, fileEncryptor, metadataConverter);
+    } catch (Exception e) {
+      aborted = true;

Review Comment:
   We should probably do the same pattern for every public method that may 
throw an exception.



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to