openinx commented on pull request #4117:
URL: https://github.com/apache/iceberg/pull/4117#issuecomment-1042701849


   Reconsidered this test case,  I think @zhongyujiang is getting the root 
cause in the correct direction.  Let's explain the cause here: 
   
   In the unit test case,  we are trying to write the following records into 
apache iceberg table by shuffling by partition field `data`  (The parallelism 
is 2): 
   
   ```
   (1, 'aaa'), (1, 'bbb'), (1, 'ccc')
   (2, 'aaa'), (2, 'bbb'), (2, 'ccc')
   (3, 'aaa'), (3, 'bbb'), (3, 'ccc')
   ```
   
   As we may produces multiple checkpoints when the streaming job is running,   
Then it's possible that we write the records in the following checkpoints: 
   
   * checkpoint#1
     * (1, 'aaa')
     * (1, 'bbb')
     * (1, 'ccc')
   
   * checkpoint#2
   
     * (2, 'aaa'), 
     * (2, 'bbb'), 
     * (2, 'ccc')
     * (3, 'aaa'), 
     * (3, 'bbb'), 
     * (3, 'ccc')
   
   Then  it will produces a seperate data file for each partition in the given 
checkpoint.  Let's say: 
   
   * checkpoint#1 
     * produces `data-file-1` for partition `aaa`
     * produces `data-file-2` for partition `bbb`
     * produces `data-file-3` for partition `ccc`
   
   * checkpoint#2
     * produces `data-file-4` for partition `aaa`
     * produces `data-file-5` for partition `bbb`
     * produces `data-file-6` for partition `ccc`
   
   Assume the snapshotState & notifyCheckpointComplete are arrived as the 
following:
   
   1. snapshotState(ckpt1);
   2. snapshotState(ckpt2);
   3. notifyCheckpointComplete(ckpt2);  ( It's possible just as the flink 
[javadoc](https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/common/state/CheckpointListener.java#L122)
 said)
   4. notifyCheckpointComplete(ckpt1);
   
   Then in the step#3,   it will commit one transaction with the alll the data 
files which comes from checkpoint#1 & checkpoint#2 (According to the this 
[IcebergFilesCommitter 
implementation](https://github.com/apache/iceberg/blob/d43cb4c296f8096300b43fe830fe6c4fb085aaa5/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java#L182))
 ,  finally this latest snapshot will include all the data files from 
`data-file-1` to `data-file-6`. That is why we encounter the failure assertion.
   
   ```
   java.lang.AssertionError: There should be 1 data file in partition 'aaa' 
expected:<1> but was:<2>
   at org.junit.Assert.fail(Assert.java:89)
   at org.junit.Assert.failNotEquals(Assert.java:835)
   at org.junit.Assert.assertEquals(Assert.java:647)
   at 
org.apache.iceberg.flink.TestFlinkTableSink.testHashDistributeMode(TestFlinkTableSink.java:283)
   ```
   


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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to