amogh-jahagirdar commented on a change in pull request #4071: URL: https://github.com/apache/iceberg/pull/4071#discussion_r830671321
########## File path: core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java ########## @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.util.Map; +import java.util.Set; + +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; + +class UpdateSnapshotReferencesOperation implements PendingUpdate<Map<String, SnapshotRef>> { + + private final TableOperations ops; + private TableMetadata base; + private final Map<String, SnapshotRef> refsToUpdate; + private final Set<String> refsToRemove; + + UpdateSnapshotReferencesOperation(TableOperations ops) { + this.ops = ops; + this.base = ops.current(); + this.refsToUpdate = Maps.newHashMap(); + this.refsToRemove = Sets.newHashSet(); + } + + @Override + public Map<String, SnapshotRef> apply() { + this.base = ops.refresh(); + Map<String, SnapshotRef> refs = Maps.newHashMap(); + TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); + for (String refName : refsToRemove) { + SnapshotRef ref = base.ref(refName); + ValidationException.check(ref != null, "Cannot remove nonexistent snapshot ref %s", refName); + if (ref.isBranch()) { + updatedBuilder.removeBranch(refName); + } + else { + updatedBuilder.removeTag(refName); + } + } + for (Map.Entry<String, SnapshotRef> refEntry : refsToUpdate.entrySet()) { + String name = refEntry.getKey(); + SnapshotRef snapshotRef = refEntry.getValue(); + updatedBuilder.createSnapshotRef(name, snapshotRef); + } + return refs; + } + + @Override + public void commit() { + Tasks.foreach(ops) + .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .exponentialBackoff( + base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .run(taskOps -> { + Map<String, SnapshotRef> refs = apply(); + TableMetadata.Builder updatedMetadata = TableMetadata.buildFrom(base); + updatedMetadata. + taskOps.commit(base, updated); + }); + } + + public UpdateSnapshotReferencesOperation createBranch(String name, long snapshotId) { + Preconditions.checkNotNull(name, "Branch name cannot be null"); + SnapshotRef branch = SnapshotRef.branchBuilder(snapshotId).build(); + refsToUpdate.put(name, branch); Review comment: Good point. If we do transaction.updateSNapshotReferenceOperations().createBranch("branch1", 1).createBranch("branch1", 2) this should fail, because independent commits would fail due to the ref already existing. To have equivalence with the chaining, it should also fail. ########## File path: core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java ########## @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.util.Map; +import java.util.Set; + +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; + +class UpdateSnapshotReferencesOperation implements PendingUpdate<Map<String, SnapshotRef>> { + + private final TableOperations ops; + private TableMetadata base; + private final Map<String, SnapshotRef> refsToUpdate; + private final Set<String> refsToRemove; + + UpdateSnapshotReferencesOperation(TableOperations ops) { + this.ops = ops; + this.base = ops.current(); + this.refsToUpdate = Maps.newHashMap(); + this.refsToRemove = Sets.newHashSet(); + } + + @Override + public Map<String, SnapshotRef> apply() { + this.base = ops.refresh(); + Map<String, SnapshotRef> refs = Maps.newHashMap(); + TableMetadata.Builder updatedBuilder = TableMetadata.buildFrom(base); + for (String refName : refsToRemove) { + SnapshotRef ref = base.ref(refName); + ValidationException.check(ref != null, "Cannot remove nonexistent snapshot ref %s", refName); + if (ref.isBranch()) { + updatedBuilder.removeBranch(refName); + } + else { + updatedBuilder.removeTag(refName); + } + } + for (Map.Entry<String, SnapshotRef> refEntry : refsToUpdate.entrySet()) { + String name = refEntry.getKey(); + SnapshotRef snapshotRef = refEntry.getValue(); + updatedBuilder.createSnapshotRef(name, snapshotRef); + } + return refs; + } + + @Override + public void commit() { + Tasks.foreach(ops) + .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .exponentialBackoff( + base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .run(taskOps -> { + Map<String, SnapshotRef> refs = apply(); + TableMetadata.Builder updatedMetadata = TableMetadata.buildFrom(base); + updatedMetadata. + taskOps.commit(base, updated); + }); + } + + public UpdateSnapshotReferencesOperation createBranch(String name, long snapshotId) { + Preconditions.checkNotNull(name, "Branch name cannot be null"); + SnapshotRef branch = SnapshotRef.branchBuilder(snapshotId).build(); + refsToUpdate.put(name, branch); Review comment: Good point. If we do transaction.updateSNapshotReferenceOperations().createBranch("branch1", 1).createBranch("branch1", 2) this should fail, because independent commits would fail due to the ref already existing. To have equivalence with the chaining, it should also fail. Will update this. -- 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