You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/31 14:01:58 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #1404: Flink: Add flink job id to state backend for handling flink job redeployment

openinx commented on a change in pull request #1404:
URL: https://github.com/apache/iceberg/pull/1404#discussion_r480150633



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
##########
@@ -108,18 +115,26 @@ public void initializeState(StateInitializationContext context) throws Exception
     this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
 
     this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR);
     if (context.isRestored()) {
-      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, flinkJobId);
-      // In the restoring path, it should have one valid snapshot for current flink job at least, so the max committed
-      // checkpoint id should be positive. If it's not positive, that means someone might have removed or expired the
-      // iceberg snapshot, in that case we should throw an exception in case of committing duplicated data files into
-      // the iceberg table.
-      Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID,

Review comment:
       Assume the case:   the flink job bootstrap first and write few records,  after the first `snapshotState(1)` finished, its job crashed, then it started to restore from the first snapshot.  Because we don't commit any iceberg transaction, so the maxCommittedCheckpointId will be -1,  finally the job won't recover successfully. 
   
   So in theory, this `Preconditions.checkState(maxCommittedCheckpointId != INITIAL_CHECKPOINT_ID)` should be incorrect here.   For the snapshot expiration case,  we may need to find other ways to deal with. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org