Skip to content

GH-3011: Deny further writes after InternalParquetRecordWriter is aborted#3450

Merged
wgtmac merged 3 commits intoapache:masterfrom
LuciferYang:GH-3011
Apr 20, 2026
Merged

GH-3011: Deny further writes after InternalParquetRecordWriter is aborted#3450
wgtmac merged 3 commits intoapache:masterfrom
LuciferYang:GH-3011

Conversation

@LuciferYang
Copy link
Copy Markdown
Contributor

@LuciferYang LuciferYang commented Mar 17, 2026

Rationale for this change

When a write fails (e.g. OOM during page flush), InternalParquetRecordWriter sets aborted = true and re-throws. But if the caller catches the exception and keeps calling write(), nothing stops them — all calls go through without error. Then close() sees aborted and skips the flush, so all those records are just gone. The caller won't know until they read the file back and find data missing.

Granted, writing after an error is the caller's fault. But silently discarding data makes it much harder to track down what went wrong. java.io.FileOutputStream handles a similar situation the same way — write() after close() is also misuse, but it still throws IOException("Stream Closed") instead of quietly accepting bytes that will never be persisted.

What changes are included in this PR?

Check aborted at the top of InternalParquetRecordWriter.write(). If aborted, throw IOException right away.

Are these changes tested?

Added testWriteAfterAbortShouldThrow in TestParquetWriterError:

  1. Write to an aborted writer → IOException with expected message
  2. close() on an aborted writer → no exception

Existing tests in parquet-hadoop still pass.

Are there any user-facing changes?

If you used to catch a write exception and keep writing to the same ParquetWriter, you'll now get an IOException on the next write(). Fix: discard the writer and create a new one.

Closes #3011

@Jiayi-Wang-db
Copy link
Copy Markdown
Contributor

Hi @LuciferYang , since we check aborted flag before flush, why would the second write produce a corrupted file?

@LuciferYang
Copy link
Copy Markdown
Contributor Author

Hi @LuciferYang , since we check aborted flag before flush, why would the second write produce a corrupted file?

Hi @Jiayi-Wang-db, thanks for the question!

You're right that the aborted check in close() prevents a corrupted file from being written — the flush is skipped, so the file won't contain malformed data.

The concern here is more about silent data loss. Consider this scenario:

  1. A write fails (e.g., OOM during page flush), and aborted is set to true.
  2. The user catches the exception but continues calling write() — these calls all succeed without error since nothing currently checks the aborted flag.
  3. When close() is called, it silently skips flushing due to the aborted flag.
  4. All records written after the failure are quietly dropped — no exception, no warning.

The user may not realize any data was lost until they read the file later and find records missing. By throwing an IOException immediately on the next write(), we make the failure explicit so the user can handle it right away rather than discovering the problem much later.

@Jiayi-Wang-db
Copy link
Copy Markdown
Contributor

Hi @LuciferYang , thanks for the clarification. Yes, it seems like it could silently swallow the exception.
However, I’m not sure if it’s the right semantic for the caller to catch an exception, simply ignore it, and continue calling write.
But since we can’t control how the caller uses it, I agree that proactively throwing the exception would be a better way to handle this case.

@wgtmac
Copy link
Copy Markdown
Member

wgtmac commented Apr 19, 2026

I agree with @Jiayi-Wang-db that once any exception reported by the writer, users should not continue to call write any more, otherwise it is a misuse.

@LuciferYang
Copy link
Copy Markdown
Contributor Author

LuciferYang commented Apr 19, 2026

Hi @wgtmac, agree — calling write after an exception is definitely a misuse. This PR just makes that misuse fail fast instead of failing silently.

Right now the writer quietly accepts all writes after abort, and close() drops them without a word. The user won't notice until they read the file and find data missing. Throwing an IOException right away at least gives them a clear signal.

java.io.FileOutputStream does the same thing — write() after close() is a misuse too, but it still throws IOException("Stream Closed") rather than silently discarding bytes.

That said, if you think this isn't worth the change, happy to close this PR and the issue. Let me know.

Copy link
Copy Markdown
Member

@wgtmac wgtmac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @LuciferYang for the quick response! I think overall the fix in this PR is benign so I'm fine with it.

@LuciferYang
Copy link
Copy Markdown
Contributor Author

Thank you @wgtmac ~

@wgtmac wgtmac merged commit 4f868ef into apache:master Apr 20, 2026
5 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Should deny further write if ParquetWriter is aborted

3 participants