Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.expressions.Term;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.Tasks;

public class BaseReplaceSortOrder implements ReplaceSortOrder {
Expand Down Expand Up @@ -57,6 +58,7 @@ public void commit() {
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
taskOps -> {
this.base = ops.refresh();
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/java/org/apache/iceberg/PropertiesUpdate.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
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.CommitRetry;
import org.apache.iceberg.util.Tasks;

class PropertiesUpdate implements UpdateProperties {
Expand Down Expand Up @@ -107,6 +108,7 @@ public void commit() {
base.propertyTryAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
taskOps -> {
Map<String, String> newProperties = apply();
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/java/org/apache/iceberg/RemoveSnapshots.java
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.SnapshotUtil;
Expand Down Expand Up @@ -366,6 +367,7 @@ public void commit() {
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
item -> {
TableMetadata updated = internalApply();
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/java/org/apache/iceberg/SetLocation.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT;

import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.Tasks;

public class SetLocation implements UpdateLocation {
Expand Down Expand Up @@ -61,6 +62,7 @@ public void commit() {
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(taskOps -> taskOps.commit(base, base.updateLocation(newLocation)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
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.util.CommitRetry;
import org.apache.iceberg.util.SnapshotUtil;
import org.apache.iceberg.util.Tasks;

Expand Down Expand Up @@ -115,6 +116,7 @@ public void commit() {
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
taskOps -> {
Snapshot snapshot = apply();
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/java/org/apache/iceberg/SetStatistics.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.Optional;
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.Tasks;

public class SetStatistics implements UpdateStatistics {
Expand Down Expand Up @@ -71,6 +72,7 @@ public void commit() {
.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
taskOps -> {
TableMetadata base = taskOps.refresh();
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/java/org/apache/iceberg/SnapshotProducer.java
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.relocated.com.google.common.math.IntMath;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.Exceptions;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.PropertyUtil;
Expand Down Expand Up @@ -494,6 +495,7 @@ public void commit() {
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.countAttempts(commitMetrics().attempts())
.run(
taskOps -> {
Expand Down
8 changes: 8 additions & 0 deletions core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.iceberg.rest;

import java.util.function.Consumer;
import org.apache.iceberg.RetryableValidationException;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.exceptions.BadRequestException;
import org.apache.iceberg.exceptions.CommitFailedException;
Expand Down Expand Up @@ -125,6 +126,13 @@ public void accept(ErrorResponse error) {
case 404:
throw new NoSuchTableException("%s", error.message());
case 409:
if (RetryableValidationException.class.getSimpleName().equals(error.type())) {
throw new CommitFailedException(
new RetryableValidationException("%s", error.message()),
"Commit failed: %s",
error.message());
}

throw new CommitFailedException("Commit failed: %s", error.message());
case 500:
case 502:
Expand Down
63 changes: 63 additions & 0 deletions core/src/main/java/org/apache/iceberg/util/CommitRetry.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.util;

import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES;
import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS;

import org.apache.iceberg.RetryableValidationException;
import org.apache.iceberg.exceptions.CommitFailedException;

public class CommitRetry {
private CommitRetry() {}

public static CommitFailedException retryExhaustedException(
Exception cause, Tasks.RetryExhaustionReason reason) {
if (isRetryableValidationCommitFailure(cause)) {
return (CommitFailedException) cause;
}

return new CommitFailedException(
cause, "Commit failed after exhausting retries. %s", retryExhaustionMessage(reason));
}

public static boolean isRetryableValidationCommitFailure(Exception cause) {
return cause instanceof CommitFailedException
&& cause.getCause() instanceof RetryableValidationException;
}

@SuppressWarnings("StatementSwitchToExpressionSwitch")
private static String retryExhaustionMessage(Tasks.RetryExhaustionReason reason) {
switch (reason) {
case ATTEMPT_LIMIT:
return String.format("To allow more retry attempts, adjust %s.", COMMIT_NUM_RETRIES);

case TIMEOUT:
return String.format(
"To allow more total retry time, adjust %s.", COMMIT_TOTAL_RETRY_TIME_MS);

case ATTEMPT_LIMIT_AND_TIMEOUT:
return String.format(
"To allow more retry attempts and total retry time, adjust %s and %s.",
COMMIT_NUM_RETRIES, COMMIT_TOTAL_RETRY_TIME_MS);
}

throw new IllegalArgumentException("Unknown retry exhaustion reason: " + reason);
}
}
87 changes: 61 additions & 26 deletions core/src/main/java/org/apache/iceberg/util/Tasks.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiFunction;
import java.util.function.Predicate;
import java.util.stream.Stream;
import org.apache.iceberg.metrics.Counter;
Expand All @@ -44,6 +45,12 @@ public class Tasks {

private Tasks() {}

public enum RetryExhaustionReason {
ATTEMPT_LIMIT,
TIMEOUT,
ATTEMPT_LIMIT_AND_TIMEOUT
}

public static class UnrecoverableException extends RuntimeException {
public UnrecoverableException(String message) {
super(message);
Expand Down Expand Up @@ -88,6 +95,8 @@ public static class Builder<I> {
private long maxDurationMs = 600000; // 10 minutes
private double scaleFactor = 2.0; // exponential
private Counter attemptsCounter;
private BiFunction<Exception, RetryExhaustionReason, RuntimeException> retryExhaustedHandler =
null;

public Builder(Iterable<I> items) {
this.items = items;
Expand Down Expand Up @@ -180,6 +189,12 @@ public Builder<I> countAttempts(Counter counter) {
return this;
}

public Builder<I> onRetryExhausted(
BiFunction<Exception, RetryExhaustionReason, RuntimeException> handler) {
this.retryExhaustedHandler = handler;
return this;
}

public Builder<I> exponentialBackoff(
long backoffMinSleepTimeMs,
long backoffMaxSleepTimeMs,
Expand Down Expand Up @@ -414,39 +429,24 @@ private <E extends Exception> void runTaskWithRetry(Task<I, E> task, I item) thr
break;

} catch (Exception e) {
long durationMs = System.currentTimeMillis() - start;
if (attempt >= maxAttempts || (durationMs > maxDurationMs && attempt > 1)) {
if (durationMs > maxDurationMs) {
LOG.info("Stopping retries after {} ms", durationMs);
}
if (!shouldRetry(e)) {
throw e;
}

if (shouldRetryPredicate != null) {
if (!shouldRetryPredicate.test(e)) {
throw e;
long durationMs = System.currentTimeMillis() - start;
boolean attemptsExhausted = attempt >= maxAttempts;
boolean timeoutExhausted = durationMs > maxDurationMs && attempt > 1;
if (attemptsExhausted || timeoutExhausted) {
if (timeoutExhausted) {
LOG.info("Stopping retries after {} ms", durationMs);
}

} else if (onlyRetryExceptions != null) {
// if onlyRetryExceptions are present, then this retries if one is found
boolean matchedRetryException = false;
for (Class<? extends Exception> exClass : onlyRetryExceptions) {
if (exClass.isInstance(e)) {
matchedRetryException = true;
break;
}
}
if (!matchedRetryException) {
throw e;
if (retryExhaustedHandler != null) {
throw retryExhaustedHandler.apply(
e, retryExhaustionReason(attemptsExhausted, timeoutExhausted));
}

} else {
// otherwise, always retry unless one of the stop exceptions is found
for (Class<? extends Exception> exClass : stopRetryExceptions) {
if (exClass.isInstance(e)) {
throw e;
}
}
throw e;
}

int delayMs =
Expand All @@ -468,6 +468,41 @@ private <E extends Exception> void runTaskWithRetry(Task<I, E> task, I item) thr
}
}
}

private boolean shouldRetry(Exception exception) {
if (shouldRetryPredicate != null) {
return shouldRetryPredicate.test(exception);
} else if (onlyRetryExceptions != null) {
// if onlyRetryExceptions are present, then this retries if one is found
for (Class<? extends Exception> exClass : onlyRetryExceptions) {
if (exClass.isInstance(exception)) {
return true;
}
}

return false;
} else {
// otherwise, always retry unless one of the stop exceptions is found
for (Class<? extends Exception> exClass : stopRetryExceptions) {
if (exClass.isInstance(exception)) {
return false;
}
}

return true;
}
}

private RetryExhaustionReason retryExhaustionReason(
boolean attemptsExhausted, boolean timeoutExhausted) {
if (attemptsExhausted && timeoutExhausted) {
return RetryExhaustionReason.ATTEMPT_LIMIT_AND_TIMEOUT;
} else if (attemptsExhausted) {
return RetryExhaustionReason.ATTEMPT_LIMIT;
} else {
return RetryExhaustionReason.TIMEOUT;
}
}
}

@SuppressWarnings("checkstyle:CyclomaticComplexity")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
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.CommitRetry;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;

Expand Down Expand Up @@ -73,6 +74,7 @@ public void commit() {
base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(taskOps -> taskOps.commit(base, internalApply()));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.iceberg.UpdateLocation;
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;

Expand Down Expand Up @@ -63,6 +64,7 @@ public void commit() {
base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(
taskOps ->
taskOps.commit(base, ViewMetadata.buildFrom(base).setLocation(apply()).build()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.util.CommitRetry;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;

Expand Down Expand Up @@ -100,6 +101,7 @@ public void commit() {
base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.onlyRetryOn(CommitFailedException.class)
.onRetryExhausted(CommitRetry::retryExhaustedException)
.run(taskOps -> taskOps.commit(base, internalApply()));
}

Expand Down
Loading