rdblue edited a comment on issue #360:
URL: https://github.com/apache/iceberg/issues/360#issuecomment-651268262


   Sounds like there is a slight difference in how we are framing the problem. 
I have been thinking of an upsert case, where we get an entire replacement row 
but not the previous row values. From your comments, it sounds like you are 
thinking of a case where we have both the previous row and the replacement row 
(possibly as separate delete and insert events). It's good to think through 
that case because it does change what we might do here.
   
   Without the previous row, I think you would have to keep track of previously 
written values, so the position deletes are attractive because you have to keep 
less information -- a position and the id columns, rather than all columns. But 
_with_ the previous row, you wouldn't need to keep any state. I agree that this 
makes it a good alternative if you have that information in your CDC stream.
   
   There's still a major problem with using equality deletes that I pointed 
out: deleting with row values would potentially delete future rows when two 
copies of a row have the same values. Using a unique ID for each CDC event 
could mitigate the problem, but duplicate events from at-least-once processing 
would still incorrectly delete data rows. How would you avoid this problem?
   
   To your point about replaying events, I think I agree that it would be a 
good to keep track of the previous column values if they are present in a CDC 
delete event, but I'm not sure it would enable a source to replay exactly the 
events that were received -- after all, we're still separating inserts and 
deletes into separate files so we can't easily reconstruct an upsert event.
   
   Assuming that it is likely valuable to track the deleted data values, I 
don't think that it makes a case for dynamic columns:
   1. It would be good to track deleted data whether the delete is encoding 
using position or equality. We can add additional columns for deleted values to 
both formats, so this doesn't strictly require equality deletes.
   2. If we introduce additional columns for deleted row values, then why 
wouldn't we add columns more to the equality delete file? They could be normal 
data files that contain deleted rows and are used differently. Then we get 
columnar encoding. Making the column list dynamic doesn't help, unless for some 
reason the CDC event is missing some columns. And in that case, it's not too 
much overhead to have a delete file for each combination of columns the CDC 
event emitter produces.
   
   ### Clarifications
   
   > the positional-deletion will only be applied to data files with the same 
sequence number.
   
   Position deletes would apply to data files with a sequence number `<=` to 
the delete file's sequence number. They would still be used for other delete 
cases, which require `<`.
   
   If we want to use equality delete files for this, then we would similarly 
apply an equality delete file when a data file's sequence number is `<=` the 
equality delete's sequence number.
   
   The optimization I was suggesting is if we don't use equality deletes to 
encode deletes within the same commit, we can use `<` instead of `<=`.
   
   > keeping the index from ID to position is expensive, especially when the 
data couldn't fit in the limited memory
   
   You could avoid spilling to disk by closing the data file when this is 
getting too expensive, instead of spilling to disk. If we assume that the ID 
column is a UUID and the position is a long, then each (id, pos) pair takes 
about 24 bytes. If we double that for JVM overhead, then we can track about 2.8 
million rows in 128 MB of memory. While that's significant, it isn't 
unreasonable to cut off data files well before 2.8 million rows. Even if the 
checkpoint interval is as long as 5 minutes, then then the rate of rows to the 
file would need to be 9,300 rows per second to exceed that limit.
   
   > The equality-deletion and positional-deletion seems make the JOIN 
algorithm complex
   
   This would happen anyway because data files will need to merge multiple 
delete files. Position deletes are more efficient because they are more 
targeted, so deployments will very likely have regular compaction from equality 
deletes to position deletes. That means at any given time, a file may have both 
delete formats to merge in. This is not that much complexity as long as we can 
filter the results of another filter.
   


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

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