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 2021/05/03 21:20:24 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #2552: Core: Fix the NPE while updating event in the context of eventual consistency.

yyanyy commented on a change in pull request #2552:
URL: https://github.com/apache/iceberg/pull/2552#discussion_r625375478



##########
File path: core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
##########
@@ -390,7 +394,16 @@ protected void validateDataFilesExist(TableMetadata base, Long startingSnapshotI
   @Override
   public Object updateEvent() {
     long snapshotId = snapshotId();
-    long sequenceNumber = ops.refresh().snapshot(snapshotId).sequenceNumber();
+    Snapshot justSaved = ops.refresh().snapshot(snapshotId);
+    long sequenceNumber = 0;
+    if( justSaved == null) {
+      // The snapshot just saved may not be present if the latest metadata couldn't be loaded due to eventual
+      // consistency problems in refresh.
+      LOG.warn("Failed to load committed snapshot, leave sequence number to 0");
+    } else {
+      sequenceNumber = justSaved.sequenceNumber();

Review comment:
       I think the fix looks good but I'm not sure if we should default `sequenceNumber` to 0, as this may result in incorrect data for users that rely on this sequence number in the event. Since without this change we probably will not send out the event at all, if we don't want to wait until the data become eventual I wonder if returning -1 would be better since 0 may refer to an actual sequence number of the table (although in v1 all sequence number will be 0 so return 0 for v1 is better, but I'm not sure how easy it is to check table version here)




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