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/18 19:59:46 UTC

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

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



##########
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 = TableMetadata.INVALID_SEQUENCE_NUMBER;

Review comment:
       Well, I am not sure what should be the best way to react here. We do expect the catalog to be consistent so one option is to just log an error and not throw an event or throw some special even type. 
   
   cc @rdsr who implemented this part and @rdblue who uses the event-based system. What do you expect in this case?




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