You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2018/03/14 20:17:42 UTC

[accumulo] branch 1.7 updated: ACCUMULO-4847 Fix Retry utility's API (#399)

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

ctubbsii pushed a commit to branch 1.7
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/1.7 by this push:
     new cf0dd9c  ACCUMULO-4847 Fix Retry utility's API (#399)
cf0dd9c is described below

commit cf0dd9c6471b0979643d913bef423efb45c5cac0
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed Mar 14 16:17:40 2018 -0400

    ACCUMULO-4847 Fix Retry utility's API (#399)
    
    Create a builder API for Retry and RetryFactory, so that the parameters
    (which all have the same type) cannot be easily confused for one another
    and cause "sleep-until-heat-death-of-the-universe" bugs.
    
    Also fix a minor bug which is caused by the assumption that nanoTime is
    always positive. It is normal for nanoTime to be negative, only the
    differences between two times is guaranteed to be positive.
---
 .../java/org/apache/accumulo/fate/util/Retry.java  | 357 +++++++++++++++++++++
 .../org/apache/accumulo/fate/zookeeper/Retry.java  | 213 ------------
 .../accumulo/fate/zookeeper/RetryFactory.java      |  74 -----
 .../apache/accumulo/fate/zookeeper/ZooReader.java  |  20 +-
 .../accumulo/fate/zookeeper/ZooReaderWriter.java   |   7 +-
 .../apache/accumulo/fate/zookeeper/ZooUtil.java    |  30 +-
 .../org/apache/accumulo/fate/util/RetryTest.java   | 294 +++++++++++++++++
 .../accumulo/fate/zookeeper/RetryFactoryTest.java  |  53 ---
 .../apache/accumulo/fate/zookeeper/RetryTest.java  | 180 -----------
 .../fate/zookeeper/ZooReaderWriterTest.java        |   4 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |  15 +-
 .../accumulo/tserver/log/TabletServerLogger.java   |  10 +-
 12 files changed, 701 insertions(+), 556 deletions(-)

diff --git a/fate/src/main/java/org/apache/accumulo/fate/util/Retry.java b/fate/src/main/java/org/apache/accumulo/fate/util/Retry.java
new file mode 100644
index 0000000..66379e9
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/util/Retry.java
@@ -0,0 +1,357 @@
+/*
+ * 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.accumulo.fate.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Encapsulates the retrying implementation for some operation. Provides bounded retry attempts with a bounded, linear backoff.
+ */
+public class Retry {
+  private static final Logger log = LoggerFactory.getLogger(Retry.class);
+
+  private long maxRetries; // not final for testing
+  private long waitIncrement; // not final for testing
+  private long maxWait; // not final for testing
+  private final long logIntervalNanoSec;
+
+  private long retriesDone;
+  private long currentWait;
+
+  private boolean hasNeverLogged;
+  private long lastRetryLog;
+
+  /**
+   * @param maxRetries
+   *          Maximum times to retry or MAX_RETRY_DISABLED if no maximum
+   * @param startWait
+   *          The amount of time (ms) to wait for the initial retry
+   * @param maxWait
+   *          The maximum wait (ms)
+   * @param waitIncrement
+   *          The amount of time (ms) to increment next wait time by
+   * @param logInterval
+   *          The amount of time (ms) between logging retries
+   */
+  private Retry(long maxRetries, long startWait, long waitIncrement, long maxWait, long logInterval) {
+    this.maxRetries = maxRetries;
+    this.maxWait = maxWait;
+    this.waitIncrement = waitIncrement;
+    this.retriesDone = 0;
+    this.currentWait = startWait;
+    this.logIntervalNanoSec = MILLISECONDS.toNanos(logInterval);
+    this.hasNeverLogged = true;
+    this.lastRetryLog = -1;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  long getMaxRetries() {
+    return maxRetries;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  long getCurrentWait() {
+    return currentWait;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  long getWaitIncrement() {
+    return waitIncrement;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  long getMaxWait() {
+    return maxWait;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  void setMaxRetries(long maxRetries) {
+    this.maxRetries = maxRetries;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  void setStartWait(long startWait) {
+    this.currentWait = startWait;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  void setWaitIncrement(long waitIncrement) {
+    this.waitIncrement = waitIncrement;
+  }
+
+  // Visible for testing
+  @VisibleForTesting
+  void setMaxWait(long maxWait) {
+    this.maxWait = maxWait;
+  }
+
+  public boolean hasInfiniteRetries() {
+    return maxRetries < 0;
+  }
+
+  public long getLogInterval() {
+    return NANOSECONDS.toMillis(logIntervalNanoSec);
+  }
+
+  public boolean canRetry() {
+    return hasInfiniteRetries() || (retriesDone < maxRetries);
+  }
+
+  public void useRetry() {
+    if (!canRetry()) {
+      throw new IllegalStateException("No retries left");
+    }
+
+    retriesDone++;
+  }
+
+  public boolean hasRetried() {
+    return retriesDone > 0;
+  }
+
+  public long retriesCompleted() {
+    return retriesDone;
+  }
+
+  public void waitForNextAttempt() throws InterruptedException {
+    log.debug("Sleeping for {}ms before retrying operation", currentWait);
+    sleep(currentWait);
+    currentWait = Math.min(maxWait, currentWait + waitIncrement);
+  }
+
+  protected void sleep(long wait) throws InterruptedException {
+    Thread.sleep(wait);
+  }
+
+  public void logRetry(Logger log, String message, Throwable t) {
+    // log the first time as debug, and then after every logInterval as a warning
+    long now = System.nanoTime();
+    if (hasNeverLogged) {
+      if (log.isDebugEnabled()) {
+        log.debug(getMessage(message, t));
+      }
+      hasNeverLogged = false;
+      lastRetryLog = now;
+    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+      log.warn(getMessage(message), t);
+      lastRetryLog = now;
+    } else {
+      if (log.isTraceEnabled()) {
+        log.trace(getMessage(message, t));
+      }
+    }
+  }
+
+  public void logRetry(Logger log, String message) {
+    // log the first time as debug, and then after every logInterval as a warning
+    long now = System.nanoTime();
+    if (hasNeverLogged) {
+      if (log.isDebugEnabled()) {
+        log.debug(getMessage(message));
+      }
+      hasNeverLogged = false;
+      lastRetryLog = now;
+    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+      log.warn(getMessage(message));
+      lastRetryLog = now;
+    } else {
+      if (log.isTraceEnabled()) {
+        log.trace(getMessage(message));
+      }
+    }
+  }
+
+  private String getMessage(String message) {
+    return message + ", retrying attempt " + (retriesDone + 1) + " (suppressing retry messages for " + getLogInterval() + "ms)";
+  }
+
+  private String getMessage(String message, Throwable t) {
+    return message + ":" + t + ", retrying attempt " + (retriesDone + 1) + " (suppressing retry messages for " + getLogInterval() + "ms)";
+  }
+
+  public interface NeedsRetries {
+    /**
+     * @return this builder with the maximum number of retries set to unlimited
+     */
+    NeedsRetryDelay infiniteRetries();
+
+    /**
+     * @param max
+     *          the maximum number of retries to set
+     * @return this builder with the maximum number of retries set to the provided value
+     */
+    NeedsRetryDelay maxRetries(long max);
+  }
+
+  public interface NeedsRetryDelay {
+    /**
+     * @param duration
+     *          the amount of time to wait before the first retry; input is converted to milliseconds, rounded down to the nearest
+     * @return this builder with the initial wait period set
+     */
+    NeedsTimeIncrement retryAfter(long duration, TimeUnit unit);
+  }
+
+  public interface NeedsTimeIncrement {
+    /**
+     * @param duration
+     *          the amount of additional time to add before each subsequent retry; input is converted to milliseconds, rounded down to the nearest
+     * @return this builder with the increment amount set
+     */
+    NeedsMaxWait incrementBy(long duration, TimeUnit unit);
+  }
+
+  public interface NeedsMaxWait {
+    /**
+     * @param duration
+     *          the maximum amount of time to which the waiting period between retries can be incremented; input is converted to milliseconds, rounded down to
+     *          the nearest
+     * @return this builder with a maximum time limit set
+     */
+    NeedsLogInterval maxWait(long duration, TimeUnit unit);
+  }
+
+  public interface NeedsLogInterval {
+    /**
+     * @param duration
+     *          the minimum time interval between logging that a retry is occurring; input is converted to milliseconds, rounded down to the nearest
+     * @return this builder with a logging interval set
+     */
+    BuilderDone logInterval(long duration, TimeUnit unit);
+  }
+
+  public interface BuilderDone {
+    /**
+     * Create a RetryFactory from this builder which can be used to create many Retry objects with the same settings.
+     *
+     * @return this builder as a factory; intermediate references to this builder cannot be used to change options after this has been called
+     */
+    RetryFactory createFactory();
+
+    /**
+     * Create a single Retry object with the currently configured builder settings.
+     *
+     * @return a retry object from this builder's settings
+     */
+    Retry createRetry();
+  }
+
+  public interface RetryFactory {
+    /**
+     * Create a single Retry object from this factory's settings.
+     *
+     * @return a retry object from this factory's settings
+     */
+    Retry createRetry();
+  }
+
+  public static NeedsRetries builder() {
+    return new RetryFactoryBuilder();
+  }
+
+  private static class RetryFactoryBuilder implements NeedsRetries, NeedsRetryDelay, NeedsTimeIncrement, NeedsMaxWait, NeedsLogInterval, BuilderDone,
+      RetryFactory {
+
+    private boolean modifiable = true;
+    private long maxRetries;
+    private long initialWait;
+    private long maxWait;
+    private long waitIncrement;
+    private long logInterval;
+
+    RetryFactoryBuilder() {}
+
+    private void checkState() {
+      Preconditions.checkState(modifiable, "Cannot modify this builder once 'createFactory()' has been called");
+    }
+
+    @Override
+    public NeedsRetryDelay infiniteRetries() {
+      checkState();
+      this.maxRetries = -1;
+      return this;
+    }
+
+    @Override
+    public NeedsRetryDelay maxRetries(long max) {
+      checkState();
+      Preconditions.checkArgument(max >= 0, "Maximum number of retries must not be negative");
+      this.maxRetries = max;
+      return this;
+    }
+
+    @Override
+    public NeedsTimeIncrement retryAfter(long duration, TimeUnit unit) {
+      checkState();
+      Preconditions.checkArgument(duration >= 0, "Initial waiting period must not be negative");
+      this.initialWait = unit.toMillis(duration);
+      return this;
+    }
+
+    @Override
+    public NeedsMaxWait incrementBy(long duration, TimeUnit unit) {
+      checkState();
+      Preconditions.checkArgument(duration >= 0, "Amount of time to increment the wait between each retry must not be negative");
+      this.waitIncrement = unit.toMillis(duration);
+      return this;
+    }
+
+    @Override
+    public NeedsLogInterval maxWait(long duration, TimeUnit unit) {
+      checkState();
+      this.maxWait = unit.toMillis(duration);
+      Preconditions.checkArgument(maxWait >= initialWait, "Maximum wait between retries must not be less than the initial delay");
+      return this;
+    }
+
+    @Override
+    public BuilderDone logInterval(long duration, TimeUnit unit) {
+      checkState();
+      Preconditions.checkArgument(duration >= 0, "The amount of time between logging retries must not be negative");
+      this.logInterval = unit.toMillis(duration);
+      return this;
+    }
+
+    @Override
+    public RetryFactory createFactory() {
+      this.modifiable = false;
+      return this;
+    }
+
+    @Override
+    public Retry createRetry() {
+      return new Retry(maxRetries, initialWait, waitIncrement, maxWait, logInterval);
+    }
+
+  }
+}
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/Retry.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/Retry.java
deleted file mode 100644
index 73d5227..0000000
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/Retry.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * 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.accumulo.fate.zookeeper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Encapsulates the retrying implementation for some operation. Provides bounded retry attempts with a bounded, linear backoff.
- */
-public class Retry {
-  private static final Logger log = LoggerFactory.getLogger(Retry.class);
-
-  public static final long MAX_RETRY_DISABLED = -1;
-
-  private long maxRetries, maxWait, waitIncrement;
-  private long retriesDone, currentWait;
-
-  private static long NANO_SEC_PER_MILLI_SEC = 1000000L;
-  private long logIntervalNanoSec;
-  private long lastRetryLog;
-
-  /**
-   * @param maxRetries
-   *          Maximum times to retry or MAX_RETRY_DISABLED if no maximum
-   * @param startWait
-   *          The amount of time (ms) to wait for the initial retry
-   * @param maxWait
-   *          The maximum wait (ms)
-   * @param waitIncrement
-   *          The amount of time (ms) to increment next wait time by
-   * @param logInterval
-   *          The amount of time (ms) between logging retries
-   */
-
-  public Retry(long maxRetries, long startWait, long waitIncrement, long maxWait, long logInterval) {
-    this.maxRetries = maxRetries;
-    this.maxWait = maxWait;
-    this.waitIncrement = waitIncrement;
-    this.retriesDone = 0l;
-    this.currentWait = startWait;
-    this.logIntervalNanoSec = (logInterval * NANO_SEC_PER_MILLI_SEC);
-    this.lastRetryLog = -1;
-  }
-
-  /**
-   * @param startWait
-   *          The amount of time (ms) to wait for the initial retry
-   * @param maxWait
-   *          The maximum wait (ms)
-   * @param waitIncrement
-   *          The amount of time (ms) to increment next wait time by
-   * @param logInterval
-   *          The amount of time (ms) between logging retries
-   */
-  public Retry(long startWait, long waitIncrement, long maxWait, long logInterval) {
-    this(MAX_RETRY_DISABLED, startWait, waitIncrement, maxWait, logInterval);
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  long getMaxRetries() {
-    return maxRetries;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  long getCurrentWait() {
-    return currentWait;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  long getWaitIncrement() {
-    return waitIncrement;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  long getMaxWait() {
-    return maxWait;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  void setMaxRetries(long maxRetries) {
-    this.maxRetries = maxRetries;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  void setStartWait(long startWait) {
-    this.currentWait = startWait;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  void setWaitIncrement(long waitIncrement) {
-    this.waitIncrement = waitIncrement;
-  }
-
-  // Visible for testing
-  @VisibleForTesting
-  void setMaxWait(long maxWait) {
-    this.maxWait = maxWait;
-  }
-
-  public boolean isMaxRetryDisabled() {
-    return maxRetries < 0;
-  }
-
-  // Visible for testing
-  void setLogInterval(long logInterval) {
-    this.logIntervalNanoSec = logInterval * NANO_SEC_PER_MILLI_SEC;
-  }
-
-  public long getLogInterval() {
-    return logIntervalNanoSec / NANO_SEC_PER_MILLI_SEC;
-  }
-
-  public boolean canRetry() {
-    return isMaxRetryDisabled() || (retriesDone < maxRetries);
-  }
-
-  public void useRetry() {
-    if (!canRetry()) {
-      throw new IllegalStateException("No retries left");
-    }
-
-    retriesDone++;
-  }
-
-  public boolean hasRetried() {
-    return retriesDone > 0;
-  }
-
-  public long retriesCompleted() {
-    return retriesDone;
-  }
-
-  public void waitForNextAttempt() throws InterruptedException {
-    if (log.isDebugEnabled()) {
-      log.debug("Sleeping for " + currentWait + "ms before retrying operation");
-    }
-    sleep(currentWait);
-    currentWait = Math.min(maxWait, currentWait + waitIncrement);
-  }
-
-  protected void sleep(long wait) throws InterruptedException {
-    Thread.sleep(wait);
-  }
-
-  public void logRetry(Logger log, String message, Throwable t) {
-    // log the first time as debug, and then after every logInterval as a warning
-    long now = System.nanoTime();
-    if (lastRetryLog < 0) {
-      if (log.isDebugEnabled()) {
-        log.debug(getMessage(message, t));
-      }
-      lastRetryLog = now;
-    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
-      log.warn(getMessage(message), t);
-      lastRetryLog = now;
-    } else {
-      if (log.isTraceEnabled()) {
-        log.trace(getMessage(message, t));
-      }
-    }
-  }
-
-  public void logRetry(Logger log, String message) {
-    // log the first time as debug, and then after every logInterval as a warning
-    long now = System.nanoTime();
-    if (lastRetryLog < 0) {
-      if (log.isDebugEnabled()) {
-        log.debug(getMessage(message));
-      }
-      lastRetryLog = now;
-    } else if ((now - lastRetryLog) > logIntervalNanoSec) {
-      log.warn(getMessage(message));
-      lastRetryLog = now;
-    } else {
-      if (log.isTraceEnabled()) {
-        log.trace(getMessage(message));
-      }
-    }
-  }
-
-  private String getMessage(String message) {
-    return message + ", retrying attempt " + (retriesDone + 1) + " (suppressing retry messages for " + getLogInterval() + "ms)";
-  }
-
-  private String getMessage(String message, Throwable t) {
-    return message + ":" + t + ", retrying attempt " + (retriesDone + 1) + " (suppressing retry messages for " + getLogInterval() + "ms)";
-  }
-
-}
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/RetryFactory.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/RetryFactory.java
deleted file mode 100644
index e56eddc..0000000
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/RetryFactory.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.accumulo.fate.zookeeper;
-
-/**
- * Creates {@link Retry} instances with the given parameters
- */
-public class RetryFactory {
-  public static final long DEFAULT_MAX_RETRIES = 10l, DEFAULT_START_WAIT = 250l, DEFAULT_WAIT_INCREMENT = 250l, DEFAULT_MAX_WAIT = 5000l;
-  // default logging interval: 3 minutes
-  public static final long DEFAULT_LOG_INTERVAL = 3 * 60 * 1000;
-  public static final RetryFactory DEFAULT_INSTANCE = new RetryFactory(DEFAULT_MAX_RETRIES, DEFAULT_START_WAIT, DEFAULT_WAIT_INCREMENT, DEFAULT_MAX_WAIT,
-      DEFAULT_LOG_INTERVAL);
-
-  private final long maxRetries, startWait, maxWait, waitIncrement, logInterval;
-
-  /**
-   * Create a retry factor for retries with a limit
-   *
-   * @param maxRetries
-   *          The maximum number of retries
-   * @param startWait
-   *          The wait ms for the first retry
-   * @param waitIncrement
-   *          The amount of ms to increment the wait on subsequent retries
-   * @param maxWait
-   *          The max amount of wait time between retries
-   * @param logInterval
-   *          The amount of time (ms) between logging retries
-   */
-  public RetryFactory(long maxRetries, long startWait, long waitIncrement, long maxWait, long logInterval) {
-    this.maxRetries = maxRetries;
-    this.startWait = startWait;
-    this.maxWait = maxWait;
-    this.waitIncrement = waitIncrement;
-    this.logInterval = logInterval;
-  }
-
-  /**
-   * Create a retry factory for retries that have no limit
-   *
-   * @param startWait
-   *          The wait ms for the first retry
-   * @param waitIncrement
-   *          The amount of ms to increment the wait on subsequent retries
-   * @param maxWait
-   *          The max amount of wait time between retries
-   */
-  public RetryFactory(long startWait, long waitIncrement, long maxWait, long logInterval) {
-    this.maxRetries = Retry.MAX_RETRY_DISABLED;
-    this.startWait = startWait;
-    this.maxWait = maxWait;
-    this.waitIncrement = waitIncrement;
-    this.logInterval = logInterval;
-  }
-
-  public Retry create() {
-    return new Retry(maxRetries, startWait, waitIncrement, maxWait, logInterval);
-  }
-}
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
index bda8307..6eed225 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
@@ -20,6 +20,8 @@ import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.ZooKeeperConnectionInfo;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.KeeperException;
@@ -69,7 +71,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public byte[] getData(String zPath, boolean watch, Stat stat) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().getData(zPath, watch, stat);
@@ -88,7 +90,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public byte[] getData(String zPath, Watcher watcher, Stat stat) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().getData(zPath, watcher, stat);
@@ -107,7 +109,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public Stat getStatus(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, false);
@@ -126,7 +128,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public Stat getStatus(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, watcher);
@@ -145,7 +147,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public List<String> getChildren(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().getChildren(zPath, false);
@@ -164,7 +166,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public List<String> getChildren(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().getChildren(zPath, watcher);
@@ -183,7 +185,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public boolean exists(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, false) != null;
@@ -202,7 +204,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public boolean exists(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, watcher) != null;
@@ -245,7 +247,7 @@ public class ZooReader implements IZooReader {
   public ZooReader(String keepers, int timeout) {
     this.keepers = keepers;
     this.timeout = timeout;
-    this.retryFactory = RetryFactory.DEFAULT_INSTANCE;
+    this.retryFactory = ZooUtil.DEFAULT_RETRY;
     this.info = new ZooKeeperConnectionInfo(keepers, timeout, null, null);
   }
 }
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
index e7ba7d5..3886ab6 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
@@ -20,6 +20,7 @@ import java.security.SecurityPermission;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.accumulo.fate.util.Retry;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.ZooKeeperConnectionInfo;
@@ -106,7 +107,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
 
   @Override
   public void delete(String path, int version) throws InterruptedException, KeeperException {
-    final Retry retry = getRetryFactory().create();
+    final Retry retry = getRetryFactory().createRetry();
     while (true) {
       try {
         getZooKeeper().delete(path, version);
@@ -137,7 +138,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
   public byte[] mutate(String zPath, byte[] createValue, List<ACL> acl, Mutator mutator) throws Exception {
     if (createValue != null) {
       while (true) {
-        final Retry retry = getRetryFactory().create();
+        final Retry retry = getRetryFactory().createRetry();
         try {
           getZooKeeper().create(zPath, createValue, acl, CreateMode.PERSISTENT);
           return createValue;
@@ -157,7 +158,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
       }
     }
     do {
-      final Retry retry = getRetryFactory().create();
+      final Retry retry = getRetryFactory().createRetry();
       Stat stat = new Stat();
       byte[] data = getData(zPath, false, stat);
       data = mutator.mutate(data);
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
index 6ea10d0..34334d8 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooUtil.java
@@ -17,13 +17,17 @@
 package org.apache.accumulo.fate.zookeeper;
 
 import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -38,6 +42,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class ZooUtil {
+
+  public static final RetryFactory DEFAULT_RETRY = Retry.builder().maxRetries(10).retryAfter(250, MILLISECONDS).incrementBy(250, MILLISECONDS)
+      .maxWait(5, TimeUnit.SECONDS).logInterval(3, TimeUnit.MINUTES).createFactory();
+
   private static final Logger log = LoggerFactory.getLogger(ZooUtil.class);
 
   public enum NodeExistsPolicy {
@@ -166,7 +174,7 @@ public class ZooUtil {
     PUBLIC = new ArrayList<>();
     PUBLIC.addAll(PRIVATE);
     PUBLIC.add(new ACL(Perms.READ, Ids.ANYONE_ID_UNSAFE));
-    RETRY_FACTORY = RetryFactory.DEFAULT_INSTANCE;
+    RETRY_FACTORY = DEFAULT_RETRY;
   }
 
   protected static ZooKeeper getZooKeeper(ZooKeeperConnectionInfo info) {
@@ -199,7 +207,7 @@ public class ZooUtil {
       throw new IllegalArgumentException(policy.name() + " is invalid for this operation");
     try {
       List<String> children;
-      final Retry retry = RETRY_FACTORY.create();
+      final Retry retry = RETRY_FACTORY.createRetry();
       while (true) {
         try {
           children = getZooKeeper(info).getChildren(zPath, false);
@@ -283,7 +291,7 @@ public class ZooUtil {
     if (policy == null)
       policy = NodeExistsPolicy.FAIL;
 
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         getZooKeeper(info).create(zPath, data, acls, mode);
@@ -329,7 +337,7 @@ public class ZooUtil {
   }
 
   public static byte[] getData(ZooKeeperConnectionInfo info, String zPath, Stat stat) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).getData(zPath, false, stat);
@@ -347,7 +355,7 @@ public class ZooUtil {
   }
 
   public static Stat getStatus(ZooKeeperConnectionInfo info, String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).exists(zPath, false);
@@ -394,7 +402,7 @@ public class ZooUtil {
       putPersistentData(info, destination, data, policy);
       if (stat.getNumChildren() > 0) {
         List<String> children;
-        final Retry retry = RETRY_FACTORY.create();
+        final Retry retry = RETRY_FACTORY.createRetry();
         while (true) {
           try {
             children = getZooKeeper(info).getChildren(source, false);
@@ -422,7 +430,7 @@ public class ZooUtil {
   }
 
   public static String putPersistentSequential(ZooKeeperConnectionInfo info, String zPath, byte[] data) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).create(zPath, data, ZooUtil.PUBLIC, CreateMode.PERSISTENT_SEQUENTIAL);
@@ -440,7 +448,7 @@ public class ZooUtil {
   }
 
   public static String putEphemeralData(ZooKeeperConnectionInfo info, String zPath, byte[] data) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).create(zPath, data, ZooUtil.PUBLIC, CreateMode.EPHEMERAL);
@@ -458,7 +466,7 @@ public class ZooUtil {
   }
 
   public static String putEphemeralSequential(ZooKeeperConnectionInfo info, String zPath, byte[] data) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).create(zPath, data, ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
@@ -492,7 +500,7 @@ public class ZooUtil {
   }
 
   public static boolean isLockHeld(ZooKeeperConnectionInfo info, LockID lid) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         List<String> children = getZooKeeper(info).getChildren(lid.path, false);
@@ -521,7 +529,7 @@ public class ZooUtil {
   }
 
   public static List<ACL> getACL(ZooKeeperConnectionInfo info, String zPath, Stat stat) throws KeeperException, InterruptedException {
-    final Retry retry = RETRY_FACTORY.create();
+    final Retry retry = RETRY_FACTORY.createRetry();
     while (true) {
       try {
         return getZooKeeper(info).getACL(zPath, stat);
diff --git a/fate/src/test/java/org/apache/accumulo/fate/util/RetryTest.java b/fate/src/test/java/org/apache/accumulo/fate/util/RetryTest.java
new file mode 100644
index 0000000..d391d3d
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/util/RetryTest.java
@@ -0,0 +1,294 @@
+/*
+ * 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.accumulo.fate.util;
+
+import static java.util.concurrent.TimeUnit.DAYS;
+import static java.util.concurrent.TimeUnit.HOURS;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.fate.util.Retry.NeedsLogInterval;
+import org.apache.accumulo.fate.util.Retry.NeedsMaxWait;
+import org.apache.accumulo.fate.util.Retry.NeedsRetries;
+import org.apache.accumulo.fate.util.Retry.NeedsRetryDelay;
+import org.apache.accumulo.fate.util.Retry.NeedsTimeIncrement;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+
+public class RetryTest {
+
+  private Retry retry;
+  private static final long INITIAL_WAIT = 1000;
+  private static final long WAIT_INC = 1000;
+  private static final long MAX_RETRIES = 5;
+  private static final long LOG_INTERVAL = 1000;
+  private Retry unlimitedRetry;
+  private static final TimeUnit MS = MILLISECONDS;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    retry = Retry.builder().maxRetries(MAX_RETRIES).retryAfter(INITIAL_WAIT, MS).incrementBy(WAIT_INC, MS).maxWait(MAX_RETRIES * WAIT_INC, MS)
+        .logInterval(LOG_INTERVAL, MS).createRetry();
+    unlimitedRetry = Retry.builder().infiniteRetries().retryAfter(INITIAL_WAIT, MS).incrementBy(WAIT_INC, MS).maxWait(MAX_RETRIES * WAIT_INC, MS)
+        .logInterval(LOG_INTERVAL, MS).createRetry();
+  }
+
+  @Test
+  public void canRetryDoesntAlterState() {
+    for (int i = 0; i < MAX_RETRIES + 1; i++) {
+      Assert.assertTrue(retry.canRetry());
+    }
+  }
+
+  @Test
+  public void hasRetriedAfterUse() {
+    Assert.assertFalse(retry.hasRetried());
+    retry.useRetry();
+    Assert.assertTrue(retry.hasRetried());
+  }
+
+  @Test
+  public void retriesAreCompleted() {
+    for (int i = 0; i < MAX_RETRIES; i++) {
+      Assert.assertEquals(i, retry.retriesCompleted());
+      // canRetry doesn't alter retry's state
+      retry.canRetry();
+      Assert.assertEquals(i, retry.retriesCompleted());
+      // Using the retry will increase the internal count
+      retry.useRetry();
+      Assert.assertEquals(i + 1, retry.retriesCompleted());
+    }
+  }
+
+  @Test
+  public void usingNonExistentRetryFails() {
+    for (int i = 0; i < MAX_RETRIES; i++) {
+      Assert.assertTrue(retry.canRetry());
+      retry.useRetry();
+    }
+    Assert.assertFalse(retry.canRetry());
+
+    // Calling useRetry when canRetry returns false throws an exception
+    exception.expect(IllegalStateException.class);
+    retry.useRetry();
+    Assert.fail("previous command should have thrown IllegalStateException");
+  }
+
+  @Test
+  public void testWaitIncrement() throws InterruptedException {
+    retry = EasyMock.createMockBuilder(Retry.class).addMockedMethod("sleep").createStrictMock();
+    retry.setMaxRetries(MAX_RETRIES);
+    retry.setStartWait(INITIAL_WAIT);
+    retry.setWaitIncrement(WAIT_INC);
+    retry.setMaxWait(MAX_RETRIES * 1000);
+
+    long currentWait = INITIAL_WAIT;
+    for (int i = 1; i <= MAX_RETRIES; i++) {
+      retry.sleep(currentWait);
+      EasyMock.expectLastCall();
+      currentWait += WAIT_INC;
+    }
+
+    EasyMock.replay(retry);
+
+    while (retry.canRetry()) {
+      retry.useRetry();
+      retry.waitForNextAttempt();
+    }
+
+    EasyMock.verify(retry);
+  }
+
+  @Test
+  public void testBoundedWaitIncrement() throws InterruptedException {
+    retry = EasyMock.createMockBuilder(Retry.class).addMockedMethod("sleep").createStrictMock();
+    retry.setMaxRetries(MAX_RETRIES);
+    retry.setStartWait(INITIAL_WAIT);
+    retry.setWaitIncrement(WAIT_INC);
+    // Make the last retry not increment in length
+    retry.setMaxWait((MAX_RETRIES - 1) * 1000);
+
+    long currentWait = INITIAL_WAIT;
+    for (int i = 1; i <= MAX_RETRIES; i++) {
+      retry.sleep(currentWait);
+      EasyMock.expectLastCall();
+      if (i < MAX_RETRIES - 1) {
+        currentWait += WAIT_INC;
+      }
+    }
+
+    EasyMock.replay(retry);
+
+    while (retry.canRetry()) {
+      retry.useRetry();
+      retry.waitForNextAttempt();
+    }
+
+    EasyMock.verify(retry);
+  }
+
+  @Test
+  public void testIsMaxRetryDisabled() {
+    Assert.assertFalse(retry.hasInfiniteRetries());
+    Assert.assertTrue(unlimitedRetry.hasInfiniteRetries());
+    Assert.assertEquals(-1, unlimitedRetry.getMaxRetries());
+  }
+
+  @Test
+  public void testUnlimitedRetry() {
+    for (int i = 0; i < Integer.MAX_VALUE; i++) {
+      Assert.assertTrue(unlimitedRetry.canRetry());
+      unlimitedRetry.useRetry();
+    }
+  }
+
+  @Test
+  public void testLogging() {
+    Logger testLogger = EasyMock.createMock(Logger.class);
+    EasyMock.expect(testLogger.isDebugEnabled()).andReturn(true);
+    testLogger.debug(EasyMock.anyObject(String.class));
+    EasyMock.expectLastCall().times(1);
+    EasyMock.expect(testLogger.isTraceEnabled()).andReturn(true).anyTimes();
+    testLogger.trace(EasyMock.anyObject(String.class));
+    EasyMock.expectLastCall().anyTimes();
+    testLogger.warn(EasyMock.anyObject(String.class));
+    EasyMock.expectLastCall().times(3, 5);
+    EasyMock.replay(testLogger);
+
+    // we want to do this for 5 second and observe the log messages
+    long start = System.currentTimeMillis();
+    long end = System.currentTimeMillis();
+    int i = 0;
+    for (; (end - start < 5000) && (i < Integer.MAX_VALUE); i++) {
+      unlimitedRetry.logRetry(testLogger, "failure message");
+      unlimitedRetry.useRetry();
+      end = System.currentTimeMillis();
+    }
+
+    // now observe what log messages we got which should be around 5 +- 1
+    EasyMock.verify(testLogger);
+    Assert.assertTrue(i > 10);
+
+  }
+
+  @Test
+  public void testMaxRetries() {
+    NeedsRetries builder = Retry.builder();
+    builder.maxRetries(10);
+    builder.maxRetries(0);
+    exception.expect(IllegalArgumentException.class);
+    builder.maxRetries(-1);
+    Assert.fail("Should not allow negative retries");
+  }
+
+  @Test
+  public void testInitialWait() {
+    NeedsRetryDelay builder = Retry.builder().maxRetries(10);
+    builder.retryAfter(10, NANOSECONDS);
+    builder.retryAfter(10, MILLISECONDS);
+    builder.retryAfter(10, DAYS);
+    builder.retryAfter(0, NANOSECONDS);
+    builder.retryAfter(0, MILLISECONDS);
+    builder.retryAfter(0, DAYS);
+
+    exception.expect(IllegalArgumentException.class);
+    builder.retryAfter(-1, NANOSECONDS);
+    Assert.fail("Should not allow negative wait times");
+  }
+
+  @Test
+  public void testIncrementBy() {
+    NeedsTimeIncrement builder = Retry.builder().maxRetries(10).retryAfter(10, MILLISECONDS);
+    builder.incrementBy(10, DAYS);
+    builder.incrementBy(10, HOURS);
+    builder.incrementBy(10, NANOSECONDS);
+    builder.incrementBy(0, DAYS);
+    builder.incrementBy(0, HOURS);
+    builder.incrementBy(0, NANOSECONDS);
+
+    exception.expect(IllegalArgumentException.class);
+    builder.incrementBy(-1, NANOSECONDS);
+    Assert.fail("Should not allow negative increments");
+  }
+
+  @Test
+  public void testMaxWait() {
+    NeedsMaxWait builder = Retry.builder().maxRetries(10).retryAfter(15, MILLISECONDS).incrementBy(10, MILLISECONDS);
+    builder.maxWait(15, MILLISECONDS);
+    builder.maxWait(16, MILLISECONDS);
+
+    exception.expect(IllegalArgumentException.class);
+    builder.maxWait(14, MILLISECONDS);
+    Assert.fail("Max wait time should be greater than or equal to initial wait time");
+  }
+
+  @Test
+  public void testLogInterval() {
+    NeedsLogInterval builder = Retry.builder().maxRetries(10).retryAfter(15, MILLISECONDS).incrementBy(10, MILLISECONDS).maxWait(16, MINUTES);
+    builder.logInterval(10, DAYS);
+    builder.logInterval(10, HOURS);
+    builder.logInterval(10, NANOSECONDS);
+    builder.logInterval(0, DAYS);
+    builder.logInterval(0, HOURS);
+    builder.logInterval(0, NANOSECONDS);
+
+    exception.expect(IllegalArgumentException.class);
+    builder.logInterval(-1, NANOSECONDS);
+    Assert.fail("Log interval must not be negative");
+  }
+
+  @Test
+  public void properArgumentsInRetry() {
+    long maxRetries = 10, startWait = 50L, maxWait = 5000L, waitIncrement = 500L, logInterval = 10000L;
+    RetryFactory factory = Retry.builder().maxRetries(maxRetries).retryAfter(startWait, MS).incrementBy(waitIncrement, MS).maxWait(maxWait, MS)
+        .logInterval(logInterval, MS).createFactory();
+    Retry retry = factory.createRetry();
+
+    Assert.assertEquals(maxRetries, retry.getMaxRetries());
+    Assert.assertEquals(startWait, retry.getCurrentWait());
+    Assert.assertEquals(maxWait, retry.getMaxWait());
+    Assert.assertEquals(waitIncrement, retry.getWaitIncrement());
+    Assert.assertEquals(logInterval, retry.getLogInterval());
+  }
+
+  @Test
+  public void properArgumentsInUnlimitedRetry() {
+    long startWait = 50L, maxWait = 5000L, waitIncrement = 500L, logInterval = 10000L;
+    RetryFactory factory = Retry.builder().infiniteRetries().retryAfter(startWait, MS).incrementBy(waitIncrement, MS).maxWait(maxWait, MS)
+        .logInterval(logInterval, MS).createFactory();
+    Retry retry = factory.createRetry();
+
+    Assert.assertEquals(-1, retry.getMaxRetries());
+    Assert.assertEquals(startWait, retry.getCurrentWait());
+    Assert.assertEquals(maxWait, retry.getMaxWait());
+    Assert.assertEquals(waitIncrement, retry.getWaitIncrement());
+    Assert.assertEquals(logInterval, retry.getLogInterval());
+  }
+
+}
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryFactoryTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryFactoryTest.java
deleted file mode 100644
index cc5d4bc..0000000
--- a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryFactoryTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.accumulo.fate.zookeeper;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- *
- */
-public class RetryFactoryTest {
-
-  @Test
-  public void properArgumentsInRetry() {
-    long maxRetries = 10, startWait = 50l, maxWait = 5000l, waitIncrement = 500l, logInterval = 10000l;
-    RetryFactory factory = new RetryFactory(maxRetries, startWait, waitIncrement, maxWait, logInterval);
-    Retry retry = factory.create();
-
-    Assert.assertEquals(maxRetries, retry.getMaxRetries());
-    Assert.assertEquals(startWait, retry.getCurrentWait());
-    Assert.assertEquals(maxWait, retry.getMaxWait());
-    Assert.assertEquals(waitIncrement, retry.getWaitIncrement());
-    Assert.assertEquals(logInterval, retry.getLogInterval());
-  }
-
-  @Test
-  public void properArgumentsInUnlimitedRetry() {
-    long startWait = 50l, maxWait = 5000l, waitIncrement = 500l, logInterval = 10000l;
-    RetryFactory factory = new RetryFactory(startWait, waitIncrement, maxWait, logInterval);
-    Retry retry = factory.create();
-
-    Assert.assertEquals(Retry.MAX_RETRY_DISABLED, retry.getMaxRetries());
-    Assert.assertEquals(startWait, retry.getCurrentWait());
-    Assert.assertEquals(maxWait, retry.getMaxWait());
-    Assert.assertEquals(waitIncrement, retry.getWaitIncrement());
-    Assert.assertEquals(logInterval, retry.getLogInterval());
-  }
-
-}
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryTest.java
deleted file mode 100644
index bca4e97..0000000
--- a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/RetryTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * 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.accumulo.fate.zookeeper;
-
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-
-/**
- *
- */
-public class RetryTest {
-
-  private Retry retry;
-  long initialWait = 1000l, waitIncrement = 1000l, maxRetries = 5, logInterval = 1000l;
-  private Retry unlimitedRetry1;
-  private Retry unlimitedRetry2;
-
-  @Before
-  public void setup() {
-    retry = new Retry(maxRetries, initialWait, waitIncrement, maxRetries * 1000l, logInterval);
-    unlimitedRetry1 = new Retry(initialWait, waitIncrement, maxRetries * 1000l, logInterval);
-    unlimitedRetry2 = new Retry(-10, initialWait, waitIncrement, maxRetries * 1000l, logInterval);
-  }
-
-  @Test
-  public void canRetryDoesntAlterState() {
-    for (int i = 0; i < maxRetries + 1; i++) {
-      Assert.assertTrue(retry.canRetry());
-    }
-  }
-
-  @Test
-  public void hasRetriedAfterUse() {
-    Assert.assertFalse(retry.hasRetried());
-    retry.useRetry();
-    Assert.assertTrue(retry.hasRetried());
-  }
-
-  @Test
-  public void retriesAreCompleted() {
-    for (int i = 0; i < maxRetries; i++) {
-      Assert.assertEquals(i, retry.retriesCompleted());
-      // canRetry doesn't alter retry's state
-      retry.canRetry();
-      Assert.assertEquals(i, retry.retriesCompleted());
-      // Using the retry will increase the internal count
-      retry.useRetry();
-      Assert.assertEquals(i + 1, retry.retriesCompleted());
-    }
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void usingNonExistentRetryFails() {
-    for (int i = 0; i < maxRetries; i++) {
-      Assert.assertTrue(retry.canRetry());
-      retry.useRetry();
-    }
-    Assert.assertFalse(retry.canRetry());
-    // Calling useRetry when canRetry returns false throws an exception
-    retry.useRetry();
-  }
-
-  @Test
-  public void testWaitIncrement() throws InterruptedException {
-    retry = EasyMock.createMockBuilder(Retry.class).addMockedMethod("sleep").createStrictMock();
-    retry.setMaxRetries(maxRetries);
-    retry.setStartWait(initialWait);
-    retry.setWaitIncrement(waitIncrement);
-    retry.setMaxWait(maxRetries * 1000l);
-
-    long currentWait = initialWait;
-    for (int i = 1; i <= maxRetries; i++) {
-      retry.sleep(currentWait);
-      EasyMock.expectLastCall();
-      currentWait += waitIncrement;
-    }
-
-    EasyMock.replay(retry);
-
-    while (retry.canRetry()) {
-      retry.useRetry();
-      retry.waitForNextAttempt();
-    }
-
-    EasyMock.verify(retry);
-  }
-
-  @Test
-  public void testBoundedWaitIncrement() throws InterruptedException {
-    retry = EasyMock.createMockBuilder(Retry.class).addMockedMethod("sleep").createStrictMock();
-    retry.setMaxRetries(maxRetries);
-    retry.setStartWait(initialWait);
-    retry.setWaitIncrement(waitIncrement);
-    // Make the last retry not increment in length
-    retry.setMaxWait((maxRetries - 1) * 1000l);
-
-    long currentWait = initialWait;
-    for (int i = 1; i <= maxRetries; i++) {
-      retry.sleep(currentWait);
-      EasyMock.expectLastCall();
-      if (i < maxRetries - 1) {
-        currentWait += waitIncrement;
-      }
-    }
-
-    EasyMock.replay(retry);
-
-    while (retry.canRetry()) {
-      retry.useRetry();
-      retry.waitForNextAttempt();
-    }
-
-    EasyMock.verify(retry);
-  }
-
-  @Test
-  public void testIsMaxRetryDisabled() {
-    Assert.assertFalse(retry.isMaxRetryDisabled());
-    Assert.assertTrue(unlimitedRetry1.isMaxRetryDisabled());
-    Assert.assertTrue(unlimitedRetry2.isMaxRetryDisabled());
-    Assert.assertEquals(Retry.MAX_RETRY_DISABLED, unlimitedRetry1.getMaxRetries());
-    Assert.assertEquals(-10, unlimitedRetry2.getMaxRetries());
-  }
-
-  @Test
-  public void testUnlimitedRetry() {
-    for (int i = 0; i < Integer.MAX_VALUE; i++) {
-      Assert.assertTrue(unlimitedRetry1.canRetry());
-      unlimitedRetry1.useRetry();
-      Assert.assertTrue(unlimitedRetry2.canRetry());
-      unlimitedRetry2.useRetry();
-    }
-  }
-
-  @Test
-  public void testLogging() {
-    Logger testLogger = EasyMock.createMock(Logger.class);
-    EasyMock.expect(testLogger.isDebugEnabled()).andReturn(true);
-    testLogger.debug(EasyMock.anyObject(String.class));
-    EasyMock.expectLastCall().times(1);
-    EasyMock.expect(testLogger.isTraceEnabled()).andReturn(true).anyTimes();
-    testLogger.trace(EasyMock.anyObject(String.class));
-    EasyMock.expectLastCall().anyTimes();
-    testLogger.warn(EasyMock.anyObject(String.class));
-    EasyMock.expectLastCall().times(3, 5);
-    EasyMock.replay(testLogger);
-
-    // we want to do this for 5 second and observe the log messages
-    long start = System.currentTimeMillis();
-    long end = System.currentTimeMillis();
-    int i = 0;
-    for (; (end - start < 5000l) && (i < Integer.MAX_VALUE); i++) {
-      unlimitedRetry1.logRetry(testLogger, "failure message");
-      unlimitedRetry1.useRetry();
-      end = System.currentTimeMillis();
-    }
-
-    // now observe what log messages we got which should be around 5 +- 1
-    EasyMock.verify(testLogger);
-    Assert.assertTrue(i > 10);
-
-  }
-}
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
index 9203b39..709da05 100644
--- a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
+++ b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
@@ -20,6 +20,8 @@ import java.lang.reflect.Method;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -53,7 +55,7 @@ public class ZooReaderWriterTest {
 
     EasyMock.expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
     EasyMock.expect(zrw.getRetryFactory()).andReturn(retryFactory).anyTimes();
-    EasyMock.expect(retryFactory.create()).andReturn(retry).anyTimes();
+    EasyMock.expect(retryFactory.createRetry()).andReturn(retry).anyTimes();
   }
 
   @Test(expected = NoNodeException.class)
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 4db28b5..1a5787b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -140,8 +140,9 @@ import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.util.LoggingRunnable;
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.RetryFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -352,13 +353,13 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     final long toleratedWalCreationFailures = aconf.getCount(Property.TSERV_WALOG_TOLERATED_CREATION_FAILURES);
     final long walFailureRetryIncrement = aconf.getTimeInMillis(Property.TSERV_WALOG_TOLERATED_WAIT_INCREMENT);
     final long walFailureRetryMax = aconf.getTimeInMillis(Property.TSERV_WALOG_TOLERATED_MAXIMUM_WAIT_DURATION);
-    // Tolerate `toleratedWalCreationFailures` failures, waiting `walFailureRetryIncrement` milliseconds after the first failure,
-    // incrementing the next wait period by the same value, for a maximum of `walFailureRetryMax` retries.
-    final RetryFactory walCreationRetryFactory = new RetryFactory(toleratedWalCreationFailures, walFailureRetryIncrement, walFailureRetryIncrement,
-        walFailureRetryMax, RetryFactory.DEFAULT_LOG_INTERVAL);
+    final RetryFactory walCreationRetryFactory = Retry.builder().maxRetries(toleratedWalCreationFailures)
+        .retryAfter(walFailureRetryIncrement, TimeUnit.MILLISECONDS).incrementBy(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
+        .maxWait(walFailureRetryMax, TimeUnit.MILLISECONDS).logInterval(3, TimeUnit.MINUTES).createFactory();
     // Tolerate infinite failures for the write, however backing off the same as for creation failures.
-    final RetryFactory walWritingRetryFactory = new RetryFactory(walFailureRetryIncrement, walFailureRetryIncrement, walFailureRetryMax,
-        RetryFactory.DEFAULT_LOG_INTERVAL);
+    final RetryFactory walWritingRetryFactory = Retry.builder().infiniteRetries().retryAfter(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
+        .incrementBy(walFailureRetryIncrement, TimeUnit.MILLISECONDS).maxWait(walFailureRetryMax, TimeUnit.MILLISECONDS).logInterval(3, TimeUnit.MINUTES)
+        .createFactory();
 
     logger = new TabletServerLogger(this, walogMaxSize, syncCounter, flushCounter, walCreationRetryFactory, walWritingRetryFactory, walogMaxAge);
     this.resourceManager = new TabletServerResourceManager(this, fs);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 1b180a2..abf5645 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -36,8 +36,8 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
-import org.apache.accumulo.fate.zookeeper.Retry;
-import org.apache.accumulo.fate.zookeeper.RetryFactory;
+import org.apache.accumulo.fate.util.Retry;
+import org.apache.accumulo.fate.util.Retry.RetryFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.replication.StatusUtil;
@@ -215,7 +215,7 @@ public class TabletServerLogger {
       return;
     } catch (Exception t) {
       if (null == createRetry) {
-        createRetry = createRetryFactory.create();
+        createRetry = createRetryFactory.createRetry();
       }
 
       // We have more retries or we exceeded the maximum number of accepted failures
@@ -267,7 +267,7 @@ public class TabletServerLogger {
   }
 
   private void write(CommitSession commitSession, boolean mincFinish, Writer writer) throws IOException {
-    write(commitSession, mincFinish, writer, writeRetryFactory.create());
+    write(commitSession, mincFinish, writer, writeRetryFactory.createRetry());
   }
 
   private void write(CommitSession commitSession, boolean mincFinish, Writer writer, Retry writeRetry) throws IOException {
@@ -276,7 +276,7 @@ public class TabletServerLogger {
   }
 
   private void write(final Collection<CommitSession> sessions, boolean mincFinish, Writer writer) throws IOException {
-    write(sessions, mincFinish, writer, writeRetryFactory.create());
+    write(sessions, mincFinish, writer, writeRetryFactory.createRetry());
   }
 
   private void write(final Collection<CommitSession> sessions, boolean mincFinish, Writer writer, Retry writeRetry) throws IOException {

-- 
To stop receiving notification emails like this one, please contact
ctubbsii@apache.org.