rdblue edited a comment on issue #360: URL: https://github.com/apache/iceberg/issues/360#issuecomment-650470573
@openinx, sorry I haven't had a chance to reply before now. We missed you at the last sync up, where we talked about this. I added this to the notes, but I wanted to follow up here as well. Sounds like there are 2 main reasons to use a dynamic schema. The first is that you might want to delete with different filters, like dropping a user by first/last name, or dropping by user id. I think it's reasonable to say there may be more than one way to delete data, but I don't think that there is much value to encoding those deletes in the same file. There is going to be a small number of column sets used to identify and delete records, and I think it's worth just using a different delete for each set of columns. Then we wouldn't need the complexity of encoding a map with various value types, and can take advantage of columnar formats. The second argument for a dynamic schema -- the CDC case with rapid upserts for the same row -- is interesting, and something I've been thinking about how to solve as well. Let me present the approach I've been thinking about. Continuing with your example, I would actually encode the delete using a positional delete rather than an equality delete. That removes the concern about deleting row with ID 1 twice because the delete would be for `(file=data_file.avro, pos=0)`. The second record would not be deleted by accident. The position-based approach has a few benefits as well: 1. Position delete files are more efficient to apply because we're using information that we already know -- exactly where the row is 2. Although we would need to keep a mapping from ID to position for every row in the data file, this would still require less memory than the approach where we encode a delete with an equality predicate with all columns of the original row. Keeping all columns of the original row is more expensive than the identity columns and an int or long position. 3. Encoding a delete using all of the columns from the previous copy of the row may delete all copies of a row. Consider the following sequence: ``` t0, UPSERT(1, 2) t1, UPSERT(1, 3) t2, UPSERT(1, 4) t3, UPSERT(1, 3) ``` The second upsert would be encoded as `id=1 and data=3`, which would actually delete the _last_ row because the data was reset. This situation could also happen if an upsert is replayed because a stream has at-least-once semantics (which should be fine, since upsert is idempotent). I think that using a positional delete is a good solution to this problem. That would also mean that we won't need to apply equality deletes to any data files with the same sequence number, which is a small win. Only positional delete files would need to be applied to data files with the same sequence number. What do you think? ---------------------------------------------------------------- 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]
