Option (2) is definitely more in line with the original design for events -- we could add a third event when the transaction is committed, and then the onus is on either the listener impl or the consumer of the listener impl's output (whatever it may be) to stitch together the events based on the transaction ID or something else.
--EM On Wed, Nov 5, 2025 at 2:54 PM Dmitri Bourlatchkov <[email protected]> wrote: > Hi Adnan, > > I agree that this is a bug. Here's the GH issue: > https://github.com/apache/polaris/issues/2990 > > I personally do _not_ think the onus for fixing this is on you :) The code > pattern that led to this issue has existed in Apache Polaris code since day > 1... but if you want to give it a try, please feel free. > > Option 1 from Yufei's email sounds like the right general approach to > fixing this problem. I also agree that actual implementation may be tricky. > It may be worth discussing an outline of a fix in this thread before > starting a PR just to avoid rework in case there's no consensus on fix > details. I'm specifically concerned about the APIs used in this case and > how Iceberg catalog transactions map to atomic Persistence operations (cf. > [1]). > > [1] https://lists.apache.org/thread/rf5orxs815zs4h64p4rwp03q3pbgxb5r > > Cheers, > Dmitri. > > On Wed, Nov 5, 2025 at 4:57 PM Adnan Hemani > <[email protected]> wrote: > > > Hi Dmitri, > > > > Yes, this is indeed a bug and we should create a GitHub issue for this to > > track fixing it. Ideally, we should be doing something similar to Option > 1 > > of what Yufei suggested - but how this will be done may be harder than it > > looks. I can investigate this and I encourage others to look into it as > > well, if this is of interest. > > > > Best, > > Adnan Hemani > > > > On Wed, Nov 5, 2025 at 8:29 AM Yufei Gu <[email protected]> wrote: > > > > > Hi Dmitri, > > > > > > Good catch! This is a subtle but important issue. Thanks for raising > it. > > I > > > think we could handle it in a few ways: > > > > > > 1. Hold the event emission until the all multi-table commit succeeds, > and > > > buffering per-table events until persistence confirms success. > > > 2. Include a transaction ID and status (e.g., pending, committed, > > aborted) > > > in emitted events so consumers can filter accordingly. This will add > > burden > > > to downstreams. I think we could figure out a way to filter out while > > > persisting them, so that the real consumers won't see the events with > > > aborted status. > > > > > > I'm not sure which way is better at this moment, we need to take a deep > > > look to evaluate both. > > > > > > Yufei > > > > > > > > > On Wed, Nov 5, 2025 at 8:06 AM Dmitri Bourlatchkov <[email protected]> > > > wrote: > > > > > > > Hi All, > > > > > > > > I'd like to highlight an aspect of the current Events behaviour with > > > > respect to multi-table commits that Christopher and I discovered > today. > > > The > > > > issue existed since the first Events implementation, AFAIK, it just > did > > > not > > > > come into focus until now, AFAIK. > > > > > > > > Consider a multi-table commit request [1]. IcebergCatalogHandler will > > run > > > > each individual table through a separate commit operation on the base > > > > catalog [2]. The base catalog will issue events for each table > > separately > > > > [3][4]. However, the overall commit to Polaris Persistence may still > > > fail, > > > > e.g. due to concurrent updates [5]. > > > > > > > > Now, we can have a situation when both before/after events are > > delivered > > > > for a table, but the actual change that triggered the events is _not_ > > > > persisted, therefore does not exist in the current state of the > Polaris > > > > catalog. > > > > > > > > Thoughts? > > > > > > > > [1] > > > > > > > > > > > > > > https://github.com/apache/polaris/blob/f934443114251f85d18c9a51ed61fc49a500a61a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java#L973 > > > > > > > > [2] > > > > > > > > > > > > > > https://github.com/apache/polaris/blob/f934443114251f85d18c9a51ed61fc49a500a61a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java#L1051 > > > > > > > > [3] > > > > > > > > > > > > > > https://github.com/apache/polaris/blob/f934443114251f85d18c9a51ed61fc49a500a61a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java#L1405 > > > > > > > > [4] > > > > > > > > > > > > > > https://github.com/apache/polaris/blob/f934443114251f85d18c9a51ed61fc49a500a61a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java#L1558 > > > > > > > > [5] > > > > > > > > > > > > > > https://github.com/apache/polaris/blob/f934443114251f85d18c9a51ed61fc49a500a61a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java#L1058 > > > > > > > > Cheers, > > > > Dmitri. > > > > > > > > > >
