szehon-ho commented on code in PR #9852: URL: https://github.com/apache/iceberg/pull/9852#discussion_r1511994436
########## core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java: ########## @@ -309,65 +304,20 @@ protected enum CommitStatus { * @return Commit Status of Success, Failure or Unknown */ protected CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) { - int maxAttempts = - PropertyUtil.propertyAsInt( - config.properties(), COMMIT_NUM_STATUS_CHECKS, COMMIT_NUM_STATUS_CHECKS_DEFAULT); - long minWaitMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_MIN_WAIT_MS, - COMMIT_STATUS_CHECKS_MIN_WAIT_MS_DEFAULT); - long maxWaitMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_MAX_WAIT_MS, - COMMIT_STATUS_CHECKS_MAX_WAIT_MS_DEFAULT); - long totalRetryMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS, - COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS_DEFAULT); - - AtomicReference<CommitStatus> status = new AtomicReference<>(CommitStatus.UNKNOWN); - - Tasks.foreach(newMetadataLocation) - .retry(maxAttempts) - .suppressFailureWhenFinished() - .exponentialBackoff(minWaitMs, maxWaitMs, totalRetryMs, 2.0) - .onFailure( - (location, checkException) -> - LOG.error("Cannot check if commit to {} exists.", tableName(), checkException)) - .run( - location -> { - TableMetadata metadata = refresh(); - String currentMetadataFileLocation = metadata.metadataFileLocation(); - boolean commitSuccess = - currentMetadataFileLocation.equals(newMetadataLocation) - || metadata.previousFiles().stream() - .anyMatch(log -> log.file().equals(newMetadataLocation)); - if (commitSuccess) { - LOG.info( - "Commit status check: Commit to {} of {} succeeded", - tableName(), - newMetadataLocation); - status.set(CommitStatus.SUCCESS); - } else { - LOG.warn( - "Commit status check: Commit to {} of {} unknown, new metadata location is not current " - + "or in history", - tableName(), - newMetadataLocation); - } - }); - - if (status.get() == CommitStatus.UNKNOWN) { - LOG.error( - "Cannot determine commit state to {}. Failed during checking {} times. " - + "Treating commit state as unknown.", - tableName(), - maxAttempts); - } - return status.get(); + return checkCommitStatus( + tableName(), newMetadataLocation, config.properties(), this::loadMetadataLocations); + } + + protected List<String> loadMetadataLocations() { + TableMetadata metadata = refresh(); + ImmutableList.Builder<String> builder = ImmutableList.builder(); + return builder + .add(metadata.metadataFileLocation()) + .addAll( + metadata.previousFiles().stream() Review Comment: I think we need to store and check previousFiles in ViewMetadata , just like we are doing for TableMetadata here. In Hive, unless we enable to provide an atomic getAndSet like in the mode provided by [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)/ https://github.com/apache/iceberg/pull/6570 , it is always possible that between commit + check, there is an intermediate commit that gets in, and the check would then fail with UNKNOWN status. (I believe it is still the correct behavior if it is UNKNOWN , ie it will not delete the metadata.json, but it is a chance that we can still return successfully for users as the commits are serialized). Can we fix this? Looks like there is already somewhere to store ViewHistoryEntry. We may need to re-use the same technique to make a common base class. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org