I don’t think it’s a problem while an alternative is explored (the JDK itself does that pretty often). So it’s up to the community: of course I’m against removing it without solid alternative, but deprecation is fine imho.
Regards JB Le mar. 19 nov. 2024 à 12:19, Ajantha Bhat <ajanthab...@gmail.com> a écrit : > - ok for deprecate equality deletes >> - not ok to remove it > > > @JB: I don't think it is a good idea to use deprecated functionality in > the new feature development. > Hence, my specific question was about kafka connect upsert operation. > > @Manu: I meant the delta writers for kafka connect Iceberg sink (which in > turn used for upsetting the CDC records) > https://github.com/apache/iceberg/issues/10842 > > > - Ajantha > > > > On Tue, Nov 19, 2024 at 3:08 PM Manu Zhang <owenzhang1...@gmail.com> > wrote: > >> I second Anton's proposal to standardize on a view-based approach to >> handle CDC cases. >> Actually, it's already been explored in detail[1] by Jack before. >> >> [1] Improving Change Data Capture Use Case for Apache Iceberg >> <https://docs.google.com/document/d/1kyyJp4masbd1FrIKUHF1ED_z1hTARL8bNoKCgb7fhSQ/edit?tab=t.0#heading=h.94xnx4qg3bnt> >> >> >> On Tue, Nov 19, 2024 at 4:16 PM Jean-Baptiste Onofré <j...@nanthrax.net> >> wrote: >> >>> My proposal is the following (already expressed): >>> - ok for deprecate equality deletes >>> - not ok to remove it >>> - work on position deletes improvements to address streaming use cases. >>> I think we should explore different approaches. Personally I think a >>> possible approach would be to find index way to data files to avoid full >>> scan to find row position. >>> >>> My $0.01 :) >>> >>> Regards >>> JB >>> >>> Le mar. 19 nov. 2024 à 07:53, Ajantha Bhat <ajanthab...@gmail.com> a >>> écrit : >>> >>>> Hi, What's the conclusion on this thread? >>>> >>>> Users are looking for Upsert (CDC) support for OSS Iceberg >>>> kafka connect sink. >>>> We only support appends at the moment. Can we go ahead and implement >>>> the upserts using equality deletes? >>>> >>>> >>>> - Ajantha >>>> >>>> On Sun, Nov 10, 2024 at 11:56 AM Vignesh <vignesh.v...@gmail.com> >>>> wrote: >>>> >>>>> Hi, >>>>> I am reading about iceberg and am quite new to this. >>>>> This puffin would be an index from key to data file. Other use cases >>>>> of Puffin, such as statistics are at a per file level if I understand >>>>> correctly. >>>>> >>>>> Where would the puffin about key->data file be stored? It is a >>>>> property of the entire table. >>>>> >>>>> Thanks, >>>>> Vignesh. >>>>> >>>>> >>>>> On Sat, Nov 9, 2024 at 2:17 AM Shani Elharrar >>>>> <sh...@upsolver.com.invalid> wrote: >>>>> >>>>>> JB, this is what we do, we write Equality Deletes and periodically >>>>>> convert them to Positional Deletes. >>>>>> >>>>>> We could probably index the keys, maybe partially index using bloom >>>>>> filters, the best would be to put those bloom filters inside puffin. >>>>>> >>>>>> Shani. >>>>>> >>>>>> On 9 Nov 2024, at 11:11, Jean-Baptiste Onofré <j...@nanthrax.net> >>>>>> wrote: >>>>>> >>>>>> >>>>>> Hi, >>>>>> >>>>>> I agree with Peter here, and I would say that it would be an issue >>>>>> for multi-engine support. >>>>>> >>>>>> I think, as I already mentioned with others, we should explore an >>>>>> alternative. >>>>>> As the main issue is the datafile scan in streaming context, maybe we >>>>>> could find a way to "index"/correlate for positional deletes with limited >>>>>> scanning. >>>>>> I will think again about that :) >>>>>> >>>>>> Regards >>>>>> JB >>>>>> >>>>>> On Sat, Nov 9, 2024 at 6:48 AM Péter Váry < >>>>>> peter.vary.apa...@gmail.com> wrote: >>>>>> >>>>>>> Hi Imran, >>>>>>> >>>>>>> I don't think it's a good idea to start creating multiple types of >>>>>>> Iceberg tables. Iceberg's main selling point is compatibility between >>>>>>> engines. If we don't have readers and writers for all types of tables, >>>>>>> then >>>>>>> we remove compatibility from the equation and engine specific formats >>>>>>> always win. OTOH, if we write readers and writers for all types of >>>>>>> tables >>>>>>> then we are back on square one. >>>>>>> >>>>>>> Identifier fields are a table schema concept and used in many cases >>>>>>> during query planning and execution. This is why they are defined as >>>>>>> part >>>>>>> of the SQL spec, and this is why Iceberg defines them as well. One use >>>>>>> case >>>>>>> is where they can be used to merge deletes (independently of how they >>>>>>> are >>>>>>> manifested) and subsequent inserts, into updates. >>>>>>> >>>>>>> Flink SQL doesn't allow creating tables with partition transforms, >>>>>>> so no new table could be created by Flink SQL using transforms, but >>>>>>> tables >>>>>>> created by other engines could still be used (both read an write). Also >>>>>>> you >>>>>>> can create such tables in Flink using the Java API. >>>>>>> >>>>>>> Requiring partition columns be part of the identifier fields is >>>>>>> coming from the practical consideration, that you want to limit the >>>>>>> scope >>>>>>> of the equality deletes as much as possible. Otherwise all of the >>>>>>> equality >>>>>>> deletes should be table global, and they should be read by every >>>>>>> reader. We >>>>>>> could write those, we just decided that we don't want to allow the user >>>>>>> to >>>>>>> do this, as it is most cases a bad idea. >>>>>>> >>>>>>> I hope this helps, >>>>>>> Peter >>>>>>> >>>>>>> On Fri, Nov 8, 2024, 22:01 Imran Rashid <iras...@cloudera.com.invalid> >>>>>>> wrote: >>>>>>> >>>>>>>> I'm not down in the weeds at all myself on implementation details, >>>>>>>> so forgive me if I'm wrong about the details here. >>>>>>>> >>>>>>>> I can see all the viewpoints -- both that equality deletes enable >>>>>>>> some use cases, but also make others far more difficult. What >>>>>>>> surprised me >>>>>>>> the most is that Iceberg does not provide a way to distinguish these >>>>>>>> two >>>>>>>> table "types". >>>>>>>> >>>>>>>> At first, I thought the presence of an identifier-field ( >>>>>>>> https://iceberg.apache.org/spec/#identifier-field-ids) indicated >>>>>>>> that the table was a target for equality deletes. But, then it turns >>>>>>>> out >>>>>>>> identifier-fields are also useful for changelog views even without >>>>>>>> equality >>>>>>>> deletes -- IIUC, they show that a delete + insert should actually be >>>>>>>> interpreted as an update in changelog view. >>>>>>>> >>>>>>>> To be perfectly honest, I'm confused about all of these details -- >>>>>>>> from my read, the spec does not indicate this relationship between >>>>>>>> identifier-fields and equality_ids in equality delete files ( >>>>>>>> https://iceberg.apache.org/spec/#equality-delete-files), but I >>>>>>>> think that is the way Flink works. Flink itself seems to have even >>>>>>>> more >>>>>>>> limitations -- no partition transforms are allowed, and all partition >>>>>>>> columns must be a subset of the identifier fields. Is that just a >>>>>>>> Flink >>>>>>>> limitation, or is that the intended behavior in the spec? (Or maybe >>>>>>>> user-error on my part?) Those seem like very reasonable limitations, >>>>>>>> from >>>>>>>> an implementation point-of-view. But OTOH, as a user, this seems to be >>>>>>>> directly contrary to some of the promises of Iceberg. >>>>>>>> >>>>>>>> Its easy to see if a table already has equality deletes in it, by >>>>>>>> looking at the metadata. But is there any way to indicate that a >>>>>>>> table (or >>>>>>>> branch of a table) _must not_ have equality deletes added to it? >>>>>>>> >>>>>>>> If that were possible, it seems like we could support both use >>>>>>>> cases. We could continue to optimize for the streaming ingestion use >>>>>>>> cases >>>>>>>> using equality deletes. But we could also build more optimizations >>>>>>>> into >>>>>>>> the "non-streaming-ingestion" branches. And we could document the >>>>>>>> tradeoff >>>>>>>> so it is much clearer to end users. >>>>>>>> >>>>>>>> To maintain compatibility, I suppose that the change would be that >>>>>>>> equality deletes continue to be allowed by default, but we'd add a new >>>>>>>> field to indicate that for some tables (or branches of a table), >>>>>>>> equality >>>>>>>> deletes would not be allowed. And it would be an error for an engine >>>>>>>> to >>>>>>>> make an update which added an equality delete to such a table. >>>>>>>> >>>>>>>> Maybe that change would even be possible in V3. >>>>>>>> >>>>>>>> And if all the performance improvements to equality deletes make >>>>>>>> this a moot point, we could drop the field in v4. But it seems like a >>>>>>>> mistake to both limit the non-streaming use-case AND have confusing >>>>>>>> limitations for the end-user in the meantime. >>>>>>>> >>>>>>>> I would happily be corrected about my understanding of all of the >>>>>>>> above. >>>>>>>> >>>>>>>> thanks! >>>>>>>> Imran >>>>>>>> >>>>>>>> On Tue, Nov 5, 2024 at 9:16 AM Bryan Keller <brya...@gmail.com> >>>>>>>> wrote: >>>>>>>> >>>>>>>>> I also feel we should keep equality deletes until we have an >>>>>>>>> alternative solution for streaming updates/deletes. >>>>>>>>> >>>>>>>>> -Bryan >>>>>>>>> >>>>>>>>> On Nov 4, 2024, at 8:33 AM, Péter Váry < >>>>>>>>> peter.vary.apa...@gmail.com> wrote: >>>>>>>>> >>>>>>>>> Well, it seems like I'm a little late, so most of the arguments >>>>>>>>> are voiced. >>>>>>>>> >>>>>>>>> I agree that we should not deprecate the equality deletes until we >>>>>>>>> have a replacement feature. >>>>>>>>> I think one of the big advantages of Iceberg is that it supports >>>>>>>>> batch processing and streaming ingestion too. >>>>>>>>> For streaming ingestion we need a way to update existing data in a >>>>>>>>> performant way, but restricting deletes for the primary keys seems >>>>>>>>> like >>>>>>>>> enough from the streaming perspective. >>>>>>>>> >>>>>>>>> Equality deletes allow a very wide range of applications, which we >>>>>>>>> might be able to narrow down a bit, but still keep useful. So if we >>>>>>>>> want to >>>>>>>>> go down this road, we need to start collecting the requirements. >>>>>>>>> >>>>>>>>> Thanks, >>>>>>>>> Peter >>>>>>>>> >>>>>>>>> Shani Elharrar <sh...@upsolver.com.invalid> ezt írta (időpont: >>>>>>>>> 2024. nov. 1., P, 19:22): >>>>>>>>> >>>>>>>>>> I understand how it makes sense for batch jobs, but it damages >>>>>>>>>> stream jobs, using equality deletes works much better for streaming >>>>>>>>>> (which >>>>>>>>>> have a strict SLA for delays), and in order to decrease the >>>>>>>>>> performance >>>>>>>>>> penalty - systems can rewrite the equality deletes to positional >>>>>>>>>> deletes. >>>>>>>>>> >>>>>>>>>> Shani. >>>>>>>>>> >>>>>>>>>> On 1 Nov 2024, at 20:06, Steven Wu <stevenz...@gmail.com> wrote: >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> Fundamentally, it is very difficult to write position deletes >>>>>>>>>> with concurrent writers and conflicts for batch jobs too, as the >>>>>>>>>> inverted >>>>>>>>>> index may become invalid/stale. >>>>>>>>>> >>>>>>>>>> The position deletes are created during the write phase. But >>>>>>>>>> conflicts are only detected at the commit stage. I assume the batch >>>>>>>>>> job >>>>>>>>>> should fail in this case. >>>>>>>>>> >>>>>>>>>> On Fri, Nov 1, 2024 at 10:57 AM Steven Wu <stevenz...@gmail.com> >>>>>>>>>> wrote: >>>>>>>>>> >>>>>>>>>>> Shani, >>>>>>>>>>> >>>>>>>>>>> That is a good point. It is certainly a limitation for the Flink >>>>>>>>>>> job to track the inverted index internally (which is what I had in >>>>>>>>>>> mind). >>>>>>>>>>> It can't be shared/synchronized with other Flink jobs or other >>>>>>>>>>> engines >>>>>>>>>>> writing to the same table. >>>>>>>>>>> >>>>>>>>>>> Thanks, >>>>>>>>>>> Steven >>>>>>>>>>> >>>>>>>>>>> On Fri, Nov 1, 2024 at 10:50 AM Shani Elharrar >>>>>>>>>>> <sh...@upsolver.com.invalid> wrote: >>>>>>>>>>> >>>>>>>>>>>> Even if Flink can create this state, it would have to be >>>>>>>>>>>> maintained against the Iceberg table, we wouldn't like duplicates >>>>>>>>>>>> (keys) if >>>>>>>>>>>> other systems / users update the table (e.g manual insert / >>>>>>>>>>>> updates using >>>>>>>>>>>> DML). >>>>>>>>>>>> >>>>>>>>>>>> Shani. >>>>>>>>>>>> >>>>>>>>>>>> On 1 Nov 2024, at 18:32, Steven Wu <stevenz...@gmail.com> >>>>>>>>>>>> wrote: >>>>>>>>>>>> >>>>>>>>>>>> >>>>>>>>>>>> > Add support for inverted indexes to reduce the cost of >>>>>>>>>>>> position lookup. This is fairly tricky to implement for streaming >>>>>>>>>>>> use cases >>>>>>>>>>>> without an external system. >>>>>>>>>>>> >>>>>>>>>>>> Anton, that is also what I was saying earlier. In Flink, the >>>>>>>>>>>> inverted index of (key, committed data files) can be tracked in >>>>>>>>>>>> Flink state. >>>>>>>>>>>> >>>>>>>>>>>> On Fri, Nov 1, 2024 at 2:16 AM Anton Okolnychyi < >>>>>>>>>>>> aokolnyc...@gmail.com> wrote: >>>>>>>>>>>> >>>>>>>>>>>>> I was a bit skeptical when we were adding equality deletes, >>>>>>>>>>>>> but nothing beats their performance during writes. We have to >>>>>>>>>>>>> find an >>>>>>>>>>>>> alternative before deprecating. >>>>>>>>>>>>> >>>>>>>>>>>>> We are doing a lot of work to improve streaming, like reducing >>>>>>>>>>>>> the cost of commits, enabling a large (potentially infinite) >>>>>>>>>>>>> number of >>>>>>>>>>>>> snapshots, changelog reads, and so on. It is a project goal to >>>>>>>>>>>>> excel in >>>>>>>>>>>>> streaming. >>>>>>>>>>>>> >>>>>>>>>>>>> I was going to focus on equality deletes after completing the >>>>>>>>>>>>> DV work. I believe we have these options: >>>>>>>>>>>>> >>>>>>>>>>>>> - Revisit the existing design of equality deletes (e.g. add >>>>>>>>>>>>> more restrictions, improve compaction, offer new writers). >>>>>>>>>>>>> - Standardize on the view-based approach [1] to handle >>>>>>>>>>>>> streaming upserts and CDC use cases, potentially making this part >>>>>>>>>>>>> of the >>>>>>>>>>>>> spec. >>>>>>>>>>>>> - Add support for inverted indexes to reduce the cost of >>>>>>>>>>>>> position lookup. This is fairly tricky to implement for streaming >>>>>>>>>>>>> use cases >>>>>>>>>>>>> without an external system. Our runtime filtering in Spark today >>>>>>>>>>>>> is >>>>>>>>>>>>> equivalent to looking up positions in an inverted index >>>>>>>>>>>>> represented by >>>>>>>>>>>>> another Iceberg table. That may still not be enough for some >>>>>>>>>>>>> streaming use >>>>>>>>>>>>> cases. >>>>>>>>>>>>> >>>>>>>>>>>>> [1] - https://www.tabular.io/blog/hello-world-of-cdc/ >>>>>>>>>>>>> >>>>>>>>>>>>> - Anton >>>>>>>>>>>>> >>>>>>>>>>>>> чт, 31 жовт. 2024 р. о 21:31 Micah Kornfield < >>>>>>>>>>>>> emkornfi...@gmail.com> пише: >>>>>>>>>>>>> >>>>>>>>>>>>>> I agree that equality deletes have their place in streaming. >>>>>>>>>>>>>> I think the ultimate decision here is how opinionated Iceberg >>>>>>>>>>>>>> wants to be >>>>>>>>>>>>>> on its use-cases. If it really wants to stick to its origins of >>>>>>>>>>>>>> "slow >>>>>>>>>>>>>> moving data", then removing equality deletes would be inline >>>>>>>>>>>>>> with this. I >>>>>>>>>>>>>> think the other high level question is how much we allow for >>>>>>>>>>>>>> partially >>>>>>>>>>>>>> compatible features (the row lineage use-case feature was >>>>>>>>>>>>>> explicitly >>>>>>>>>>>>>> approved excluding equality deletes, and people seemed OK with >>>>>>>>>>>>>> it at the >>>>>>>>>>>>>> time. If all features need to work together, then maybe we need >>>>>>>>>>>>>> to rethink >>>>>>>>>>>>>> the design here so it can be forward compatible with equality >>>>>>>>>>>>>> deletes). >>>>>>>>>>>>>> >>>>>>>>>>>>>> I think one issue with equality deletes as stated in the spec >>>>>>>>>>>>>> is that they are overly broad. I'd be interested if people have >>>>>>>>>>>>>> any use >>>>>>>>>>>>>> cases that differ, but I think one way of narrowing (and >>>>>>>>>>>>>> probably a >>>>>>>>>>>>>> necessary building block for building something better) the >>>>>>>>>>>>>> specification >>>>>>>>>>>>>> scope on equality deletes is to focus on upsert/Streaming >>>>>>>>>>>>>> deletes. Two >>>>>>>>>>>>>> proposals in this regard are: >>>>>>>>>>>>>> >>>>>>>>>>>>>> 1. Require that equality deletes can only correspond to >>>>>>>>>>>>>> unique identifiers for the table. >>>>>>>>>>>>>> 2. Consider requiring that for equality deletes on >>>>>>>>>>>>>> partitioned tables, that the primary key must contain a >>>>>>>>>>>>>> partition column (I >>>>>>>>>>>>>> believe Flink at least already does this). It is less clear to >>>>>>>>>>>>>> me that >>>>>>>>>>>>>> this would meet all existing use-cases. But having this would >>>>>>>>>>>>>> allow for >>>>>>>>>>>>>> better incremental data-structures, which could then be >>>>>>>>>>>>>> partition based. >>>>>>>>>>>>>> >>>>>>>>>>>>>> Narrow scope to unique identifiers would allow for further >>>>>>>>>>>>>> building blocks already mentioned, like a secondary index >>>>>>>>>>>>>> (possible via LSM >>>>>>>>>>>>>> tree), that would allow for better performance overall. >>>>>>>>>>>>>> >>>>>>>>>>>>>> I generally agree with the sentiment that we shouldn't >>>>>>>>>>>>>> deprecate them until there is a viable replacement. With all >>>>>>>>>>>>>> due respect >>>>>>>>>>>>>> to my employer, let's not fall into the Google trap [1] :) >>>>>>>>>>>>>> >>>>>>>>>>>>>> Cheers, >>>>>>>>>>>>>> Micah >>>>>>>>>>>>>> >>>>>>>>>>>>>> [1] https://goomics.net/50/ >>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>>>>>>> >>>>>>>>>>>>>> On Thu, Oct 31, 2024 at 12:35 PM Alexander Jo < >>>>>>>>>>>>>> alex...@starburstdata.com> wrote: >>>>>>>>>>>>>> >>>>>>>>>>>>>>> Hey all, >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Just to throw my 2 cents in, I agree with Steven and others >>>>>>>>>>>>>>> that we do need some kind of replacement before deprecating >>>>>>>>>>>>>>> equality >>>>>>>>>>>>>>> deletes. >>>>>>>>>>>>>>> They certainly have their problems, and do significantly >>>>>>>>>>>>>>> increase complexity as they are now, but the writing of >>>>>>>>>>>>>>> position deletes is >>>>>>>>>>>>>>> too expensive for certain pipelines. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> We've been investigating using equality deletes for some of >>>>>>>>>>>>>>> our workloads at Starburst, the key advantage we were hoping to >>>>>>>>>>>>>>> leverage is >>>>>>>>>>>>>>> cheap, effectively random access lookup deletes. >>>>>>>>>>>>>>> Say you have a UUID column that's unique in a table and want >>>>>>>>>>>>>>> to delete a row by UUID. With position deletes each delete is >>>>>>>>>>>>>>> expensive >>>>>>>>>>>>>>> without an index on that UUID. >>>>>>>>>>>>>>> With equality deletes each delete is cheap and while >>>>>>>>>>>>>>> reads/compaction is expensive but when updates are frequent and >>>>>>>>>>>>>>> reads are >>>>>>>>>>>>>>> sporadic that's a reasonable tradeoff. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Pretty much what Jason and Steven have already said. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Maybe there are some incremental improvements on equality >>>>>>>>>>>>>>> deletes or tips from similar systems that might alleviate some >>>>>>>>>>>>>>> of their >>>>>>>>>>>>>>> problems? >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> - Alex Jo >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> On Thu, Oct 31, 2024 at 10:58 AM Steven Wu < >>>>>>>>>>>>>>> stevenz...@gmail.com> wrote: >>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> We probably all agree with the downside of equality >>>>>>>>>>>>>>>> deletes: it postpones all the work on the read path. >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> In theory, we can implement position deletes only in the >>>>>>>>>>>>>>>> Flink streaming writer. It would require the tracking of last >>>>>>>>>>>>>>>> committed >>>>>>>>>>>>>>>> data files per key, which can be stored in Flink state >>>>>>>>>>>>>>>> (checkpointed). This >>>>>>>>>>>>>>>> is obviously quite expensive/challenging, but possible. >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> I like to echo one benefit of equality deletes that Russel >>>>>>>>>>>>>>>> called out in the original email. Equality deletes would never >>>>>>>>>>>>>>>> have conflicts. that is important for streaming writers >>>>>>>>>>>>>>>> (Flink, Kafka >>>>>>>>>>>>>>>> connect, ...) that commit frequently (minutes or less). Assume >>>>>>>>>>>>>>>> Flink can >>>>>>>>>>>>>>>> write position deletes only and commit every 2 minutes. The >>>>>>>>>>>>>>>> long-running >>>>>>>>>>>>>>>> nature of streaming jobs can cause frequent commit conflicts >>>>>>>>>>>>>>>> with >>>>>>>>>>>>>>>> background delete compaction jobs. >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> Overall, the streaming upsert write is not a well solved >>>>>>>>>>>>>>>> problem in Iceberg. This probably affects all streaming >>>>>>>>>>>>>>>> engines (Flink, >>>>>>>>>>>>>>>> Kafka connect, Spark streaming, ...). We need to come up with >>>>>>>>>>>>>>>> some better >>>>>>>>>>>>>>>> alternatives before we can deprecate equality deletes. >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> On Thu, Oct 31, 2024 at 8:38 AM Russell Spitzer < >>>>>>>>>>>>>>>> russell.spit...@gmail.com> wrote: >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> For users of Equality Deletes, what are the key >>>>>>>>>>>>>>>>> benefits to Equality Deletes that you would like to preserve >>>>>>>>>>>>>>>>> and could you >>>>>>>>>>>>>>>>> please share some concrete examples of the queries you want >>>>>>>>>>>>>>>>> to run (and the >>>>>>>>>>>>>>>>> schemas and data sizes you would like to run them against) >>>>>>>>>>>>>>>>> and the >>>>>>>>>>>>>>>>> latencies that would be acceptable? >>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> On Thu, Oct 31, 2024 at 10:05 AM Jason Fine >>>>>>>>>>>>>>>>> <ja...@upsolver.com.invalid> wrote: >>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> Hi, >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> Representing Upsolver here, we also make use of Equality >>>>>>>>>>>>>>>>>> Deletes to deliver high frequency low latency updates to our >>>>>>>>>>>>>>>>>> clients at >>>>>>>>>>>>>>>>>> scale. We have customers using them at scale and >>>>>>>>>>>>>>>>>> demonstrating the need and >>>>>>>>>>>>>>>>>> viability. We automate the process of converting them into >>>>>>>>>>>>>>>>>> positional >>>>>>>>>>>>>>>>>> deletes (or fully applying them) for more efficient engine >>>>>>>>>>>>>>>>>> queries in the >>>>>>>>>>>>>>>>>> background giving our users both low latency and good query >>>>>>>>>>>>>>>>>> performance. >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> Equality Deletes were added since there isn't a good way >>>>>>>>>>>>>>>>>> to solve frequent updates otherwise. It would require some >>>>>>>>>>>>>>>>>> sort of index >>>>>>>>>>>>>>>>>> keeping track of every record in the table (by a >>>>>>>>>>>>>>>>>> predetermined PK) and >>>>>>>>>>>>>>>>>> maintaining such an index is a huge task that every tool >>>>>>>>>>>>>>>>>> interested in this >>>>>>>>>>>>>>>>>> would need to re-implement. It also becomes a bottleneck >>>>>>>>>>>>>>>>>> limiting table >>>>>>>>>>>>>>>>>> sizes. >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> I don't think they should be removed without providing an >>>>>>>>>>>>>>>>>> alternative. Positional Deletes have a different performance >>>>>>>>>>>>>>>>>> profile >>>>>>>>>>>>>>>>>> inherently, requiring more upfront work proportional to the >>>>>>>>>>>>>>>>>> table size. >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> On Thu, Oct 31, 2024 at 2:45 PM Jean-Baptiste Onofré < >>>>>>>>>>>>>>>>>> j...@nanthrax.net> wrote: >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> Hi Russell >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> Thanks for the nice writeup and the proposal. >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> I agree with your analysis, and I have the same feeling. >>>>>>>>>>>>>>>>>>> However, I >>>>>>>>>>>>>>>>>>> think there are more than Flink that write equality >>>>>>>>>>>>>>>>>>> delete files. So, >>>>>>>>>>>>>>>>>>> I agree to deprecate in V3, but maybe be more "flexible" >>>>>>>>>>>>>>>>>>> about removal >>>>>>>>>>>>>>>>>>> in V4 in order to give time to engines to update. >>>>>>>>>>>>>>>>>>> I think that by deprecating equality deletes, we are >>>>>>>>>>>>>>>>>>> clearly focusing >>>>>>>>>>>>>>>>>>> on read performance and "consistency" (more than write). >>>>>>>>>>>>>>>>>>> It's not >>>>>>>>>>>>>>>>>>> necessarily a bad thing but the streaming platform and >>>>>>>>>>>>>>>>>>> data ingestion >>>>>>>>>>>>>>>>>>> platforms will be probably concerned about that (by >>>>>>>>>>>>>>>>>>> using positional >>>>>>>>>>>>>>>>>>> deletes, they will have to scan/read all datafiles to >>>>>>>>>>>>>>>>>>> find the >>>>>>>>>>>>>>>>>>> position, so painful). >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> So, to summarize: >>>>>>>>>>>>>>>>>>> 1. Agree to deprecate equality deletes, but -1 to commit >>>>>>>>>>>>>>>>>>> any target >>>>>>>>>>>>>>>>>>> for deletion before having a clear path for streaming >>>>>>>>>>>>>>>>>>> platforms >>>>>>>>>>>>>>>>>>> (Flink, Beam, ...) >>>>>>>>>>>>>>>>>>> 2. In the meantime (during the deprecation period), I >>>>>>>>>>>>>>>>>>> propose to >>>>>>>>>>>>>>>>>>> explore possible improvements for streaming platforms >>>>>>>>>>>>>>>>>>> (maybe finding a >>>>>>>>>>>>>>>>>>> way to avoid full data files scan, ...) >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> Thanks ! >>>>>>>>>>>>>>>>>>> Regards >>>>>>>>>>>>>>>>>>> JB >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>> On Wed, Oct 30, 2024 at 10:06 PM Russell Spitzer >>>>>>>>>>>>>>>>>>> <russell.spit...@gmail.com> wrote: >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Background: >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > 1) Position Deletes >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Writers determine what rows are deleted and mark them >>>>>>>>>>>>>>>>>>> in a 1 for 1 representation. With delete vectors this means >>>>>>>>>>>>>>>>>>> every data file >>>>>>>>>>>>>>>>>>> has at most 1 delete vector that it is read in conjunction >>>>>>>>>>>>>>>>>>> with to excise >>>>>>>>>>>>>>>>>>> deleted rows. Reader overhead is more or less constant and >>>>>>>>>>>>>>>>>>> is very >>>>>>>>>>>>>>>>>>> predictable. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > The main cost of this mode is that deletes must be >>>>>>>>>>>>>>>>>>> determined at write time which is expensive and can be more >>>>>>>>>>>>>>>>>>> difficult for >>>>>>>>>>>>>>>>>>> conflict resolution >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > 2) Equality Deletes >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Writers write out reference to what values are deleted >>>>>>>>>>>>>>>>>>> (in a partition or globally). There can be an unlimited >>>>>>>>>>>>>>>>>>> number of equality >>>>>>>>>>>>>>>>>>> deletes and they all must be checked for every data file >>>>>>>>>>>>>>>>>>> that is read. The >>>>>>>>>>>>>>>>>>> cost of determining deleted rows is essentially given to >>>>>>>>>>>>>>>>>>> the reader. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Conflicts almost never happen since data files are not >>>>>>>>>>>>>>>>>>> actually changed and there is almost no cost to the writer >>>>>>>>>>>>>>>>>>> to generate >>>>>>>>>>>>>>>>>>> these. Almost all costs related to equality deletes are >>>>>>>>>>>>>>>>>>> passed on to the >>>>>>>>>>>>>>>>>>> reader. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Proposal: >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Equality deletes are, in my opinion, unsustainable and >>>>>>>>>>>>>>>>>>> we should work on deprecating and removing them from the >>>>>>>>>>>>>>>>>>> specification. At >>>>>>>>>>>>>>>>>>> this time, I know of only one engine (Apache Flink) which >>>>>>>>>>>>>>>>>>> produces these >>>>>>>>>>>>>>>>>>> deletes but almost all engines have implementations to read >>>>>>>>>>>>>>>>>>> them. The cost >>>>>>>>>>>>>>>>>>> of implementing equality deletes on the read path is >>>>>>>>>>>>>>>>>>> difficult and >>>>>>>>>>>>>>>>>>> unpredictable in terms of memory usage and compute >>>>>>>>>>>>>>>>>>> complexity. We’ve had >>>>>>>>>>>>>>>>>>> suggestions of implementing rocksdb inorder to handle ever >>>>>>>>>>>>>>>>>>> growing sets of >>>>>>>>>>>>>>>>>>> equality deletes which in my opinion shows that we are >>>>>>>>>>>>>>>>>>> going down the wrong >>>>>>>>>>>>>>>>>>> path. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Outside of performance, Equality deletes are also >>>>>>>>>>>>>>>>>>> difficult to use in conjunction with many other features. >>>>>>>>>>>>>>>>>>> For example, any >>>>>>>>>>>>>>>>>>> features requiring CDC or Row lineage are basically >>>>>>>>>>>>>>>>>>> impossible when >>>>>>>>>>>>>>>>>>> equality deletes are in use. When Equality deletes are >>>>>>>>>>>>>>>>>>> present, the state >>>>>>>>>>>>>>>>>>> of the table can only be determined with a full scan making >>>>>>>>>>>>>>>>>>> it difficult to >>>>>>>>>>>>>>>>>>> update differential structures. This means materialized >>>>>>>>>>>>>>>>>>> views or indexes >>>>>>>>>>>>>>>>>>> need to essentially be fully rebuilt whenever an equality >>>>>>>>>>>>>>>>>>> delete is added >>>>>>>>>>>>>>>>>>> to the table. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Equality deletes essentially remove complexity from >>>>>>>>>>>>>>>>>>> the write side but then add what I believe is an >>>>>>>>>>>>>>>>>>> unacceptable level of >>>>>>>>>>>>>>>>>>> complexity to the read side. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Because of this I suggest we deprecate Equality >>>>>>>>>>>>>>>>>>> Deletes in V3 and slate them for full removal from the >>>>>>>>>>>>>>>>>>> Iceberg Spec in V4. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > I know this is a big change and compatibility breakage >>>>>>>>>>>>>>>>>>> so I would like to introduce this idea to the community and >>>>>>>>>>>>>>>>>>> solicit >>>>>>>>>>>>>>>>>>> feedback from all stakeholders. I am very flexible on this >>>>>>>>>>>>>>>>>>> issue and would >>>>>>>>>>>>>>>>>>> like to hear the best issues both for and against removal >>>>>>>>>>>>>>>>>>> of Equality >>>>>>>>>>>>>>>>>>> Deletes. >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Thanks everyone for your time, >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > Russ Spitzer >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> > >>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> -- >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> *Jason Fine* >>>>>>>>>>>>>>>>>> Chief Software Architect >>>>>>>>>>>>>>>>>> ja...@upsolver.com | www.upsolver.com >>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> >>>>>>>>>