hbgstc123 commented on PR #7595:
URL: https://github.com/apache/paimon/pull/7595#issuecomment-4591585819

   > This is a critical data safety fix. The data loss scenario during sort 
compact with concurrent writes is well-analyzed.
   > 
   > **The timeline demonstrating the bug is clear and scary**: file D gets 
deleted without its data appearing in the sorted output.
   > 
   > **Solution review:**
   > 
   > 1. **Part 1 (Pin DELETE to base snapshot)**: Correct. The DELETE list 
should only contain files that were actually read and included in the sorted 
output. Building from `latestSnapshot` is fundamentally unsafe with concurrent 
writers.
   > 2. **Part 2 (Concurrent write detection)**: Using 
`readIncrementalChanges(baseSnapshot, latestSnapshot)` to detect new files in 
the overwritten partitions, then failing with conflict error — this is the 
right approach. Fail-loud is better than data loss.
   > 3. **556 additions** across core commit logic, Flink actions, and Spark 
writer. This is appropriately scoped for a correctness fix that spans the 
commit protocol.
   > 
   > **Questions:**
   > 
   > 4. Is the conflict detection granularity at partition level or file level? 
Partition-level might cause false conflicts (concurrent write to same partition 
but different bucket). File-level would be more precise but more complex.
   > 5. What happens for non-partitioned tables? The "overwritten partitions" 
concept doesn't apply — is the entire table treated as one partition?
   > 6. The test `SortCompactActionForAppendTableITCase` — does it reproduce 
the exact race condition (concurrent write between read and commit)?
   > 
   > This should be high priority for merge. Silent data loss is the worst 
class of bug.
   
   Thanks for the review!
   
   For the questions:
   4.Detection granularity. It is implemented at partition level in this PR, 
because I think allowing other writer to write into the overwriting partition 
does not honor overwrite-partition semantics. We can solve the false conflict 
issue in a new method that don't named "overwritePartition". What do yo think?
   
   5.For non-partition table. It is treated as one partition, and I just added 
some test to cover it.
   
   6.I have change the test to cover the case the PR desc, is that what you 
mean?
   
   


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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to