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
9 changes: 9 additions & 0 deletions .palantir/revapi.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1655,3 +1655,12 @@ acceptedBreaks:
\ boolean)"
justification: "IncrementalScanEvent should only be constructed by Iceberg code.\
\ Hence the change of constructor params shouldn't affect users"
"1.11.0":
org.apache.iceberg:iceberg-core:
- code: "java.method.removed"
old: "method org.apache.iceberg.util.Tasks.Builder<I> org.apache.iceberg.util.Tasks.Builder<I>::exponentialBackoff(long,\
\ long, long, double)"
justification: "Split into Tasks.Builder.totalTimeoutMs(long) for the retry\
\ budget and Tasks.Builder.backoffStrategy(BackoffStrategy) for the per-attempt\
\ wait formula. The previous 4-arg method conflated two unrelated concerns\
\ and obscured operator-facing terminology (commit.retry.total-timeout-ms)."
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,8 @@ public boolean acquire(String entityId, String ownerId) {
Tasks.foreach(entityId)
.throwFailureWhenFinished()
.retry(Integer.MAX_VALUE - 1)
.exponentialBackoff(acquireIntervalMs(), acquireIntervalMs(), acquireTimeoutMs(), 1)
.totalTimeoutMs(acquireTimeoutMs())
.backoffStrategy(backoffStrategy())
.onlyRetryOn(
ConditionalCheckFailedException.class,
ProvisionedThroughputExceededException.class,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import org.apache.iceberg.util.BackoffStrategies;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand Down Expand Up @@ -121,7 +122,8 @@ protected CommitStatus checkCommitStatusStrict(
Tasks.foreach(newMetadataLocation)
.retry(maxAttempts)
.suppressFailureWhenFinished()
.exponentialBackoff(minWaitMs, maxWaitMs, totalRetryMs, 2.0)
.totalTimeoutMs(totalRetryMs)
.backoffStrategy(BackoffStrategies.from(properties, minWaitMs, maxWaitMs, 2.0))
.onFailure(
(location, checkException) ->
LOG.error("Cannot check if commit to {} exists.", tableOrViewName, checkException))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.relocated.com.google.common.base.Objects;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.util.BackoffStrategies;
import org.apache.iceberg.util.LocationUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand Down Expand Up @@ -193,7 +194,10 @@ protected void refreshFromMetadataLocation(
AtomicReference<TableMetadata> newMetadata = new AtomicReference<>();
Tasks.foreach(newLocation)
.retry(numRetries)
.exponentialBackoff(100, 5000, 600000, 4.0 /* 100, 400, 1600, ... */)
.totalTimeoutMs(600000)
.backoffStrategy(
BackoffStrategies.from(
currentMetadata != null ? currentMetadata.properties() : null, 100, 5000, 4.0))
.throwFailureWhenFinished()
.stopRetryOn(NotFoundException.class) // overridden if shouldRetry is non-null
.shouldRetryTest(shouldRetry)
Expand Down
14 changes: 9 additions & 5 deletions core/src/main/java/org/apache/iceberg/BaseReplaceSortOrder.java
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.BackoffStrategies;
import org.apache.iceberg.util.Tasks;

public class BaseReplaceSortOrder implements ReplaceSortOrder {
Expand All @@ -51,11 +52,14 @@ public SortOrder apply() {
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down
31 changes: 19 additions & 12 deletions core/src/main/java/org/apache/iceberg/BaseTransaction.java
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.BackoffStrategies;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand Down Expand Up @@ -301,14 +302,17 @@ private void commitReplaceTransaction(boolean orCreate) {
try {
Tasks.foreach(ops)
.retry(PropertyUtil.propertyAsInt(props, COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
.exponentialBackoff(
.totalTimeoutMs(
PropertyUtil.propertyAsInt(
props, COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
PropertyUtil.propertyAsInt(
props, COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
PropertyUtil.propertyAsInt(
props, COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
props, COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
props,
PropertyUtil.propertyAsInt(
props, COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
PropertyUtil.propertyAsInt(
props, COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
underlyingOps -> {
Expand Down Expand Up @@ -359,11 +363,14 @@ private void commitSimpleTransaction() {
try {
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
underlyingOps -> {
Expand Down
14 changes: 9 additions & 5 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.BackoffStrategies;
import org.apache.iceberg.util.Tasks;

class PropertiesUpdate implements UpdateProperties {
Expand Down Expand Up @@ -101,11 +102,14 @@ public void commit() {
// If existing table commit properties in base are corrupted, allow rectification
Tasks.foreach(ops)
.retry(base.propertyTryAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
.exponentialBackoff(
base.propertyTryAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
base.propertyTryAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
base.propertyTryAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.totalTimeoutMs(
base.propertyTryAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
base.propertyTryAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
base.propertyTryAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down
14 changes: 9 additions & 5 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.BackoffStrategies;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.SnapshotUtil;
Expand Down Expand Up @@ -356,11 +357,14 @@ private Set<Long> unreferencedSnapshotsToRetain(Collection<SnapshotRef> refs) {
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
item -> {
Expand Down
14 changes: 9 additions & 5 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.BackoffStrategies;
import org.apache.iceberg.util.Tasks;

public class SetLocation implements UpdateLocation {
Expand Down Expand Up @@ -55,11 +56,14 @@ public void commit() {
TableMetadata base = ops.refresh();
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(taskOps -> taskOps.commit(base, base.updateLocation(newLocation)));
}
Expand Down
14 changes: 9 additions & 5 deletions core/src/main/java/org/apache/iceberg/SetSnapshotOperation.java
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.BackoffStrategies;
import org.apache.iceberg.util.SnapshotUtil;
import org.apache.iceberg.util.Tasks;

Expand Down Expand Up @@ -109,11 +110,14 @@ public Snapshot apply() {
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down
16 changes: 11 additions & 5 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.BackoffStrategies;
import org.apache.iceberg.util.Tasks;

public class SetStatistics implements UpdateStatistics {
Expand Down Expand Up @@ -64,12 +65,17 @@ public List<StatisticsFile> apply() {
public void commit() {
Tasks.foreach(ops)
.retry(ops.current().propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT))
.exponentialBackoff(
ops.current().propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
ops.current().propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
.totalTimeoutMs(
ops.current()
.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT),
2.0 /* exponential */)
.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
ops.current().properties(),
ops.current()
.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT),
ops.current()
.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT),
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down
14 changes: 9 additions & 5 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.BackoffStrategies;
import org.apache.iceberg.util.Exceptions;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.PropertyUtil;
Expand Down Expand Up @@ -463,11 +464,14 @@ public void commit() {
try {
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 */)
.totalTimeoutMs(
base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT))
.backoffStrategy(
BackoffStrategies.from(
base.properties(),
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),
2.0))
.onlyRetryOn(CommitFailedException.class)
.countAttempts(commitMetrics().attempts())
.run(
Expand Down
25 changes: 15 additions & 10 deletions core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@
import org.apache.iceberg.rest.responses.LoadViewResponse;
import org.apache.iceberg.rest.responses.PlanTableScanResponse;
import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse;
import org.apache.iceberg.util.BackoffStrategies;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.view.BaseView;
Expand Down Expand Up @@ -615,11 +616,13 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
try {
Tasks.foreach(ops)
.retry(COMMIT_NUM_RETRIES_DEFAULT)
.exponentialBackoff(
COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
2.0 /* exponential */)
.totalTimeoutMs(COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT)
.backoffStrategy(
BackoffStrategies.from(
ops.current() != null ? ops.current().properties() : null,
COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down Expand Up @@ -777,11 +780,13 @@ static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
try {
Tasks.foreach(ops)
.retry(COMMIT_NUM_RETRIES_DEFAULT)
.exponentialBackoff(
COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
2.0 /* exponential */)
.totalTimeoutMs(COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT)
.backoffStrategy(
BackoffStrategies.from(
ops.current() != null ? ops.current().properties() : null,
COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
2.0))
.onlyRetryOn(CommitFailedException.class)
.run(
taskOps -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.apache.iceberg.rest.responses.FetchPlanningResultResponse;
import org.apache.iceberg.rest.responses.PlanTableScanResponse;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.util.BackoffStrategies;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand Down Expand Up @@ -261,7 +262,9 @@ private CloseableIterable<FileScanTask> fetchPlanningResult() {
AtomicReference<FetchPlanningResultResponse> result = new AtomicReference<>();
try {
Tasks.foreach(planId)
.exponentialBackoff(MIN_SLEEP_MS, MAX_SLEEP_MS, maxWaitTimeMs, SCALE_FACTOR)
.totalTimeoutMs(maxWaitTimeMs)
.backoffStrategy(
BackoffStrategies.from(catalogProperties, MIN_SLEEP_MS, MAX_SLEEP_MS, SCALE_FACTOR))
.retry(MAX_RETRIES)
.onlyRetryOn(NotCompleteException.class)
.onFailure(
Expand Down
Loading