Skip to content

feat: Support Iceberg's single-table multi-statement writes transaction#25003

Open
hantangwangd wants to merge 9 commits intoprestodb:masterfrom
hantangwangd:pass_autocommit_context_to_connector
Open

feat: Support Iceberg's single-table multi-statement writes transaction#25003
hantangwangd wants to merge 9 commits intoprestodb:masterfrom
hantangwangd:pass_autocommit_context_to_connector

Conversation

@hantangwangd
Copy link
Member

@hantangwangd hantangwangd commented Apr 29, 2025

Description

Iceberg currently supports transactions at the table level via the single-table transaction API, which allows one or more updates to a single table in an atomic manner. For example:

    Transaction txn = table.newTransaction();
    txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
    txn.newDelete().deleteFile(FILE_A).commit();
    txn.newAppend().appendFile(FILE_D).commit();
    txn.commitTransaction();

Meanwhile, the Iceberg community is also working on supporting multi-table multi-write transactions, as referenced in this document and PR. According to the discussion in the document, the support of multi-statement read-write transactions should be a catalog-level implementation, and it also covers the choice of isolation levels and the definition of related behaviors in Iceberg. Currently, the implementation appears not so mature to support multi-table operations particularly at the SERIALIZABLE isolation level. However, its behavioral definition and implementation details for the SNAPSHOT isolation level can be fully applied to a catalog-based single-table transaction involving multiple write statements.

Based on this, this PR implements single-table multi-statement writes transactions in Presto Iceberg Connector, with the following design choices:

  1. Supports multi-statement writes on a single table, while can read for multiple tables in the same transaction. All operations are guaranteed isolation-level visibility and atomic commit/rollback.

  2. Currently, only REPEATABLE_READ(SNAPSHOT) isolation level is supported, as this is the most natural/easiest isolation level to implement at this stage. The SERIALIZABLE isolation level seems to have some remaining concerns -- we'd need to ensure it properly handles concurrent cases. Also, it would likely require a lot of code changes. Once the Iceberg community's implementation of multi-table transactions with SERIALIZABLE isolation matures, we can follow up with support for this isolation level as well.

  3. Following the discussion in the Iceberg community's design documentation, we implement the following behavior to guarantee SNAPSHOT isolation level in multi-statement read-write transactions:

  • Load requirement:
All reads that are being made will see the last committed values that existed when the table was loaded first inside the catalog transaction.
Subsequent changes to a table that happened outside the catalog transaction after the table was read will not be seen to prevent read skew.
  • Commit requirement:
Will successfully commit only if the values updated by the transaction do not conflict with other concurrent updates.

Note that under SNAPSHOT isolation a write skew anomaly is acceptable and permitted.

  1. Supports read your own writes visibility within transactions, meaning subsequent operations can see the results of previous writes within the same transaction.

  2. Within multi-statement transactions, only operations manageable through Iceberg's single-table transaction API are supported. Certain operations (e.g. CREATE TABLE/DROP TABLE/RENAME TABLE etc.) cannot be managed through Iceberg's single-table transaction API. Therefore, currently they are unsuitable to be included in multi-statement transactions and should instead be executed in autocommit mode (autocommit=true). We referenced the test cases from Iceberg community's implementation here to determine which statements should be supported in a multi-statement read-write transaction.


Besides, due to limitations in the current testing framework, our test cases do not currently include validation of transaction isolation under concurrent execution. Once we adopt the enhancement for testing framework from PR #25053, we will be able to write the following test cases to test concurrent transactions:

assertUpdate("create table multi_statements_transaction_commit(a int, b varchar)");
assertUpdate("insert into multi_statements_transaction_commit values(1, '1001')", 1);

Session sessionWithTx1 = assertStartTransaction(getSession(), "start transaction");
Session sessionWithTx2 = assertStartTransaction(getSession(), "start transaction");

assertUpdate(sessionWithTx1, "insert into multi_statements_transaction_commit values(2, '1002')", 1);
assertQuery(sessionWithTx1, "select * from multi_statements_transaction_commit", "values(1, '1001'), (2, '1002')");
assertUpdate(sessionWithTx2, "insert into multi_statements_transaction_commit values(3, '1003')", 1);
assertQuery(sessionWithTx2, "select * from multi_statements_transaction_commit", "values(1, '1001'), (3, '1003')");
assertQuery("select * from multi_statements_transaction_commit", "values(1, '1001')");

// `Commit` transaction1 successfully
session = assertEndTransaction(sessionWithTx1, "commit");
assertQuery("select * from multi_statements_transaction_commit", "values(1, '1001'), (2, '1002')");
assertQuery(sessionWithTx2, "select * from multi_statements_transaction_commit", "values(1, '1001'), (3, '1003')");

// `Commit` transaction1 successfully
session = assertEndTransaction(sessionWithTx2, "commit");
assertQuery("select * from multi_statements_transaction_commit", "values(1, '1001'), (2, '1002'), (3, '1003')");

assertUpdate(session, "drop table if exists multi_statements_transaction_commit");

Motivation and Context

Support single-table multi-statement writes transaction in Iceberg Connector

Impact

  • Iceberg Connector now supports single-table multi-statement writes transaction

Test Plan

  • Newly added test cases for single-table multi-statement transaction

Contributor checklist

  • Please make sure your submission complies with our contributing guide, in particular code style and commit standards.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.
== RELEASE NOTES ==

SPI Changes
* Update SPI method `Connector.beginTransaction` in a backward compatible way to support passing the autocommit context into connector transactions

Iceberg Connector Changes
* Add single-table multi-statement writes transaction on snapshot isolation level.

@hantangwangd hantangwangd marked this pull request as ready for review April 29, 2025 14:36
@hantangwangd hantangwangd requested review from a team and ZacBlanco as code owners April 29, 2025 14:36
@tdcmeehan
Copy link
Contributor

I am a bit confused about the intent of this PR. Could you help to provide some examples on a scenario in Iceberg where it is helpful to add this flag? I note that the isSingleStatementWritesOnly flag is still true in this PR for the Iceberg connector.

I also don't see where the autoCommitContext is actually used in this PR. What changes once this PR is merged?

@tdcmeehan tdcmeehan self-assigned this May 19, 2025
@hantangwangd
Copy link
Member Author

@tdcmeehan, thanks for the review and comment. I think the core purpose of this PR is to support connectors that allow multi-write transaction for a subset of SQL statements. Currently, for many connectors (like Iceberg), multi-write statement transaction support is typically limited to some certain DML operations, with little to no support for DDL statements. It's not an all-or-nothing proposition, so a connector-level isSingleStatementWritesOnly flag appears insufficient for this purpose. We need a way to let each connector determine - based on the current transaction's autoCommitContext flag - whether a given statement should be permitted execution within the current transaction.

Take Iceberg for example, with this autoCommitContext flag passed in, it can mark it's connector level flag isSingleStatementWritesOnly as false, and explicitly check and throw an exception when execute DDL statements like DROP TABLE in a transaction with flag autoCommitContext=false.

I also don't see where the autoCommitContext is actually used in this PR. What changes once this PR is merged?

This PR do not really open the multi-write statement transaction for Iceberg. After this is merged, I plan to support the actual multi-statements write transaction for Iceberg in one or several follow-up PRs. What's your opinion?

@tdcmeehan
Copy link
Contributor

@hantangwangd can you include the Iceberg changes into this PR as a separate commit?

@hantangwangd
Copy link
Member Author

can you include the Iceberg changes into this PR as a separate commit?

Sure, I'll add the change later. My thought is, in the beginning, we can allow DQL and some certain DML to run together within a transaction with autoCommitContext=false, under the snapshot isolation. Then, we can progressively expand the range of statements permitted in multi-write statement transactions based on the specific characteristics of each Iceberg operation. Do you think this makes sense? @tdcmeehan

@hantangwangd hantangwangd marked this pull request as draft May 26, 2025 16:18
@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 4 times, most recently from 10386b3 to c440544 Compare May 28, 2025 07:05
@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 3 times, most recently from 55028af to a4fc379 Compare July 23, 2025 18:37
@hantangwangd hantangwangd changed the title Pass autocommit context to connector on starting a transaction [Iceberg] Support single-table multi-statement writes transaction Jul 24, 2025
@hantangwangd hantangwangd marked this pull request as ready for review July 24, 2025 06:03
@hantangwangd
Copy link
Member Author

Hi @tdcmeehan, I've added the code changes to support multi-statement writes transactions for Iceberg, and added the relevant design choices to the PR description. Please take a look when convenient. Any feedback would be greatly appreciated!

@hantangwangd hantangwangd requested a review from yingsu00 July 24, 2025 07:34
@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 2 times, most recently from 61383e0 to 574b3ac Compare September 1, 2025 07:25
@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 2 times, most recently from f79248e to d7e430c Compare November 27, 2025 01:35
@steveburnett
Copy link
Contributor

Please resolve the file conflicts to move this PR closer to merging.

@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch from 4fea82c to b889d72 Compare December 10, 2025 07:43
@hantangwangd
Copy link
Member Author

@steveburnett thank you for the reminder. Done! Please take a look when you get a chance.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

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

Looks really great. I only have some nits.

private final boolean autoCommitContext;
private final Map<SchemaTableName, Transaction> txByTable;
private final Map<SchemaTableName, Table> initiallyReadTables;
private Optional<Runnable> callbacksOnCommit = Optional.empty();
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
private Optional<Runnable> callbacksOnCommit = Optional.empty();
private final AtomicReference<Runnable> callbacksOnCommit = Optional.empty();

Copy link
Member Author

Choose a reason for hiding this comment

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

callbacksOnCommit isn't a final field–it need to be set in the below registerCallback method.

Copy link
Member Author

Choose a reason for hiding this comment

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

@tdcmeehan sorry, just noticed this now — I initially misread your suggestion as final Optional<Runnable>. As you recommended, using final AtomicReference<Runnable> is undoubtedly a better choice, and I've already fixed it. Please take a look when you have a moment. Thanks a lot!

@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch from b889d72 to 6a4eb25 Compare December 11, 2025 03:13
@hantangwangd
Copy link
Member Author

@tdcmeehan thanks a lot for your review. I've addressed the comments, please take another look when you get a chance.

@steveburnett
Copy link
Contributor

steveburnett commented Jan 8, 2026

@steveburnett thank you for the reminder. Done! Please take a look when you get a chance.

Just getting back to this, apologies for the delay. Almost all the tests have passed, barring one that was canceled.

Please resolve the file conflicts that have developed over the time, and then we can find how the tests are doing now!

@steveburnett
Copy link
Contributor

Just a ping that this PR has file conflicts needing resolution, but otherwise seems close to mergeability.

@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 2 times, most recently from 92a9cfb to 51ab9ff Compare January 30, 2026 07:44
@hantangwangd
Copy link
Member Author

@steveburnett thanks for your reminder. I've rebased and resolved the conflicts, please take a look when you get a chance.

@steveburnett
Copy link
Contributor

@steveburnett thanks for your reminder. I've rebased and resolved the conflicts, please take a look when you get a chance.

And there's another file conflict now, in IcebergAbstractMetadata.java. When you have time, please resolve this new conflict to move the PR towards mergeability.

@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch 3 times, most recently from 85a70e8 to 30137d9 Compare February 18, 2026 14:40
@hantangwangd
Copy link
Member Author

Thanks @steveburnett. I've rebased and resolved the conflicts. Please take a look when you get a chance.

@steveburnett
Copy link
Contributor

Thanks @steveburnett. I've rebased and resolved the conflicts. Please take a look when you get a chance.

Looks good! I see no file conflict, and all checks have passed.

@hantangwangd hantangwangd force-pushed the pass_autocommit_context_to_connector branch from 30137d9 to e7631a9 Compare February 26, 2026 17:15
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.

3 participants