You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sh...@apache.org on 2020/04/29 07:46:43 UTC

[incubator-ratis] branch master updated: RATIS-882. Implement ExponentialBackoffRetry. (#63)

This is an automated email from the ASF dual-hosted git repository.

shashikant pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new babc3d3  RATIS-882. Implement ExponentialBackoffRetry. (#63)
babc3d3 is described below

commit babc3d3ffae4f623b5a7b804b4b7c7ec29db3830
Author: Lokesh Jain <lj...@apache.org>
AuthorDate: Wed Apr 29 13:16:32 2020 +0530

    RATIS-882. Implement ExponentialBackoffRetry. (#63)
---
 .../ratis/retry/ExponentialBackoffRetry.java       | 90 ++++++++++++++++++++++
 .../java/org/apache/ratis/util/Preconditions.java  |  9 +++
 .../ratis/retry/TestExponentialBackoffRetry.java   | 81 +++++++++++++++++++
 3 files changed, 180 insertions(+)

diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java
new file mode 100644
index 0000000..be7c5ed
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.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.ratis.retry;
+
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TimeDuration;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Retry Policy exponentially increases sleep time with randomness on successive
+ * retries. The sleep time is a geometric progression b*2, b*4, b*8, b*16...
+ * bounded by maximum configured duration.
+ *
+ * If sleep time calculated using the progression is s then randomness is added
+ * in the range [s*0.5, s*1.5).
+ */
+public class ExponentialBackoffRetry implements RetryPolicy {
+
+  public static class Builder {
+
+    private Builder() {}
+
+    private TimeDuration baseSleepTime;
+    private TimeDuration maxSleepTime = null;
+    private int maxAttempts = Integer.MAX_VALUE;
+
+    public Builder setMaxAttempts(int maxAttempts) {
+      this.maxAttempts = maxAttempts;
+      return this;
+    }
+
+    public Builder setBaseSleepTime(TimeDuration baseSleepTime) {
+      this.baseSleepTime = baseSleepTime;
+      return this;
+    }
+
+    public Builder setMaxSleepTime(TimeDuration maxSleepTime) {
+      this.maxSleepTime = maxSleepTime;
+      return this;
+    }
+
+    public ExponentialBackoffRetry build() {
+      Preconditions.assertNotNull(baseSleepTime, "baseSleepTime");
+      return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime,
+          maxAttempts);
+    }
+  }
+
+  private final TimeDuration baseSleepTime;
+  private final TimeDuration maxSleepTime;
+  private final int maxAttempts;
+
+  private ExponentialBackoffRetry(TimeDuration baseSleepTime, TimeDuration maxSleepTime, int maxAttempts) {
+    this.baseSleepTime = baseSleepTime;
+    this.maxSleepTime = maxSleepTime;
+    this.maxAttempts = maxAttempts;
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  private TimeDuration getSleepTime(int attemptCount) {
+    TimeDuration sleepTime = baseSleepTime.multiply(Math.pow(2, attemptCount));
+    sleepTime = maxSleepTime != null && sleepTime.compareTo(maxSleepTime) > 0 ? maxSleepTime : sleepTime;
+    return sleepTime.multiply(ThreadLocalRandom.current().nextDouble() + 0.5);
+  }
+
+  @Override
+  public Action handleAttemptFailure(Event event) {
+    TimeDuration sleepTime = getSleepTime(event.getAttemptCount());
+    return event.getAttemptCount() < maxAttempts ? () -> sleepTime : NO_RETRY_ACTION;
+  }
+}
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
index 337a5d4..fbad32d 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
@@ -90,6 +90,15 @@ public interface Preconditions {
         + name + " = " + object + " != null, class = " + object.getClass());
   }
 
+  static void assertNotNull(Object object, Supplier<String> message) {
+    assertTrue(object != null, message);
+  }
+
+  static void assertNotNull(Object object, String name) {
+    assertNotNull(object, () -> name + " is expected to not be null but "
+        + name + " = " + object + " == null, class = " + object.getClass());
+  }
+
   static <T> void assertUnique(Iterable<T> first) {
     assertUnique(first, Collections.emptyList());
   }
diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java
new file mode 100644
index 0000000..232077e
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java
@@ -0,0 +1,81 @@
+/*
+ * 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.ratis.retry;
+
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.util.TimeDuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests ExponentialBackoffRetry policy.
+ */
+public class TestExponentialBackoffRetry extends BaseTest {
+
+  @Test
+  public void testExponentialBackoffRetry() {
+    TimeDuration baseSleep = TimeDuration.valueOf(1, TimeUnit.SECONDS);
+    TimeDuration maxSleep = TimeDuration.valueOf(40, TimeUnit.SECONDS);
+
+    // Test maxAttempts
+    ExponentialBackoffRetry retryPolicy = createPolicy(baseSleep, null, 1);
+    Assert.assertFalse(retryPolicy.handleAttemptFailure(() -> 1).shouldRetry());
+
+    try {
+      // baseSleep should not be null
+      createPolicy(null, null, 1);
+      Assert.fail("Policy creation should have failed");
+    } catch (Throwable t) {
+    }
+
+    // test policy without max sleep
+    retryPolicy = createPolicy(baseSleep, null,100);
+    assertSleep(retryPolicy, baseSleep, null);
+
+    // test policy with max sleep
+    retryPolicy = createPolicy(baseSleep, maxSleep,100);
+    assertSleep(retryPolicy, baseSleep, maxSleep);
+  }
+
+  private void assertSleep(ExponentialBackoffRetry retryPolicy,
+      TimeDuration baseSleep, TimeDuration maxSleep) {
+    for (int i = 1; i <= 50; i++) {
+      int attempt = i;
+      RetryPolicy.Action action = retryPolicy.handleAttemptFailure(() -> attempt);
+
+      // sleep time based on geometric progresssion
+      long d = (1L << attempt) * baseSleep.toLong(TimeUnit.MILLISECONDS);
+      d = Math.min(d, maxSleep != null ? maxSleep.toLong(TimeUnit.MILLISECONDS) : Long.MAX_VALUE);
+
+      // sleep time with randomness added
+      long randomizedDuration = action.getSleepTime().toLong(TimeUnit.MILLISECONDS);
+
+      Assert.assertTrue(action.shouldRetry());
+      Assert.assertTrue(randomizedDuration >= d * 0.5);
+      Assert.assertTrue(randomizedDuration <= d * 1.5);
+    }
+  }
+
+  private ExponentialBackoffRetry createPolicy(TimeDuration baseSleep,
+      TimeDuration maxSleep, int maxAttempts) {
+    return ExponentialBackoffRetry.newBuilder().setBaseSleepTime(baseSleep)
+        .setMaxAttempts(maxAttempts).setMaxSleepTime(maxSleep).build();
+  }
+}