[NO ISSUE]: Fix TokenAcquireTimeout and introduce Backoff for retries
- user model changes: no
- storage format changes: no
- interface changes: no
details:
- Fix token acquire timeout from 5sec to 100ms
- Add a Full Jittered Exponential Backoff for retries
Ext-ref: MB-65063
Change-Id: I1c2a1ba129d725e09d7bca2ef44ecdeebe82eda0
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19376
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ritik Raj <raj.ritik9835@gmail.com>
diff --git a/hyracks-fullstack/hyracks/hyracks-cloud/pom.xml b/hyracks-fullstack/hyracks/hyracks-cloud/pom.xml
index 6ff2ef6..07dcacc 100644
--- a/hyracks-fullstack/hyracks/hyracks-cloud/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-cloud/pom.xml
@@ -87,5 +87,15 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-core</artifactId>
+ <version>${gcsjavasdk.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>sdk-core</artifactId>
+ <version>${awsjavasdk.version}</version>
+ </dependency>
</dependencies>
</project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/util/CloudRetryableRequestUtil.java b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/util/CloudRetryableRequestUtil.java
index 2d0afbe..be8be9c 100644
--- a/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/util/CloudRetryableRequestUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-cloud/src/main/java/org/apache/hyracks/cloud/util/CloudRetryableRequestUtil.java
@@ -25,9 +25,15 @@
import org.apache.hyracks.cloud.io.request.ICloudBeforeRetryRequest;
import org.apache.hyracks.cloud.io.request.ICloudRequest;
import org.apache.hyracks.cloud.io.request.ICloudReturnableRequest;
+import org.apache.hyracks.util.ExponentialRetryPolicy;
+import org.apache.hyracks.util.IRetryPolicy;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import com.google.cloud.BaseServiceException;
+
+import software.amazon.awssdk.core.exception.SdkException;
+
/**
* Run {@link ICloudRequest} and {@link ICloudReturnableRequest} with retries
*/
@@ -39,10 +45,13 @@
* @see System#setProperty(String, String)
*/
public static final String CLOUD_UNSTABLE_MODE = "cloud.unstable.mode";
- private static final int STABLE_NUMBER_OF_RETRIES = 5;
+ private static final int STABLE_NUMBER_OF_RETRIES = 10;
+ private static final long STABLE_MAX_DELAY_BETWEEN_RETRIES_IN_MILLIS = 10_000;
private static final int UNSTABLE_NUMBER_OF_RETRIES = 100;
+ private static final int UNSTABLE_MAX_DELAY_BETWEEN_RETRIES_IN_MILLIS = 0;
private static final Logger LOGGER = LogManager.getLogger();
private static final int NUMBER_OF_RETRIES = getNumberOfRetries();
+ private static final long MAX_DELAY_BETWEEN_RETRIES = getMaxDelayBetweenRetries();
private static final ICloudBeforeRetryRequest NO_OP_RETRY = () -> {
};
@@ -162,11 +171,15 @@
private static <T> T doRun(ICloudReturnableRequest<T> request, ICloudBeforeRetryRequest retry)
throws HyracksDataException {
int attempt = 1;
+ IRetryPolicy retryPolicy = null;
while (true) {
try {
return request.call();
- } catch (IOException e) {
- if (attempt > NUMBER_OF_RETRIES) {
+ } catch (IOException | BaseServiceException | SdkException e) {
+ if (retryPolicy == null) {
+ retryPolicy = new ExponentialRetryPolicy(NUMBER_OF_RETRIES, MAX_DELAY_BETWEEN_RETRIES);
+ }
+ if (!retryPolicy.retry(e)) {
throw HyracksDataException.create(e);
}
attempt++;
@@ -179,12 +192,16 @@
private static void doRun(ICloudRequest request, ICloudBeforeRetryRequest retry) throws HyracksDataException {
int attempt = 1;
+ IRetryPolicy retryPolicy = null;
while (true) {
try {
request.call();
break;
- } catch (IOException e) {
- if (attempt > NUMBER_OF_RETRIES) {
+ } catch (IOException | BaseServiceException | SdkException e) {
+ if (retryPolicy == null) {
+ retryPolicy = new ExponentialRetryPolicy(NUMBER_OF_RETRIES, MAX_DELAY_BETWEEN_RETRIES);
+ }
+ if (!retryPolicy.retry(e)) {
throw HyracksDataException.create(e);
}
attempt++;
@@ -199,4 +216,8 @@
return unstable ? UNSTABLE_NUMBER_OF_RETRIES : STABLE_NUMBER_OF_RETRIES;
}
+ private static long getMaxDelayBetweenRetries() {
+ boolean unstable = Boolean.getBoolean(CLOUD_UNSTABLE_MODE);
+ return unstable ? UNSTABLE_MAX_DELAY_BETWEEN_RETRIES_IN_MILLIS : STABLE_MAX_DELAY_BETWEEN_RETRIES_IN_MILLIS;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExponentialRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExponentialRetryPolicy.java
new file mode 100644
index 0000000..a5255ab
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExponentialRetryPolicy.java
@@ -0,0 +1,90 @@
+/*
+ * 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.hyracks.util;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+public class ExponentialRetryPolicy implements IRetryPolicy {
+
+ private static final int DEFAULT_MAX_RETRIES = 10;
+ private static final long DEFAULT_INITIAL_DELAY_IN_MILLIS = 100;
+ private static final long DEFAULT_MAX_DELAY_IN_MILLIS = Long.MAX_VALUE;
+ private final int maxRetries;
+ private final long initialDelay;
+ private final long maxDelay;
+ private int attempt = 0;
+
+ /**
+ * Default constructor for ExponentialRetryPolicy.
+ * Initializes with default max retries, initial delay, and max delay.
+ */
+ public ExponentialRetryPolicy() {
+ this(DEFAULT_MAX_RETRIES, DEFAULT_INITIAL_DELAY_IN_MILLIS, DEFAULT_MAX_DELAY_IN_MILLIS);
+ }
+
+ /**
+ * ExponentialRetryPolicy with specified max retries, initial delay, and max delay.
+ *
+ * @param maxRetries the maximum number of retries
+ * @param initialDelay the initial delay in milliseconds
+ * @param maxDelay the maximum delay in milliseconds
+ */
+ public ExponentialRetryPolicy(int maxRetries, long initialDelay, long maxDelay) {
+ this.maxRetries = maxRetries;
+ this.initialDelay = initialDelay;
+ this.maxDelay = maxDelay;
+ }
+
+ /**
+ * ExponentialRetryPolicy with specified max retries.
+ * Initializes with default initial delay and max delay.
+ *
+ * @param maxRetries the maximum number of retries
+ */
+ public ExponentialRetryPolicy(int maxRetries) {
+ this(maxRetries, DEFAULT_INITIAL_DELAY_IN_MILLIS, DEFAULT_MAX_DELAY_IN_MILLIS);
+ }
+
+ /**
+ * ExponentialRetryPolicy with specified max retries and max delay.
+ * Initializes with default initial delay.
+ *
+ * @param maxRetries the maximum number of retries
+ * @param maxDelay the maximum delay in milliseconds
+ */
+ public ExponentialRetryPolicy(int maxRetries, long maxDelay) {
+ this(maxRetries, DEFAULT_INITIAL_DELAY_IN_MILLIS, maxDelay);
+ }
+
+ @Override
+ public boolean retry(Throwable failure) {
+ if (attempt < maxRetries) {
+ try {
+ long delay = initialDelay * (1L << attempt);
+ TimeUnit.MILLISECONDS.sleep(ThreadLocalRandom.current().nextLong(1 + Long.min(delay, maxDelay)));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ attempt++;
+ return true;
+ }
+ return false;
+ }
+}