Added RetryingFunction for simplified retry support.
Moved retry logic out of primitives (AtomicCounter) to the caller site.

Change-Id: I319d61f153f98d421baf32a1b5cd69d20dc63427
diff --git a/core/api/src/main/java/org/onosproject/store/service/AtomicCounterBuilder.java b/core/api/src/main/java/org/onosproject/store/service/AtomicCounterBuilder.java
index e43d7dd..41a19f0 100644
--- a/core/api/src/main/java/org/onosproject/store/service/AtomicCounterBuilder.java
+++ b/core/api/src/main/java/org/onosproject/store/service/AtomicCounterBuilder.java
@@ -15,8 +15,6 @@
  */
 package org.onosproject.store.service;
 
-import java.util.concurrent.ScheduledExecutorService;
-
 /**
  * Builder for AtomicCounter.
  */
@@ -50,15 +48,6 @@
     AtomicCounterBuilder withPartitionsDisabled();
 
     /**
-     * Enables retries when counter operations fail.
-     * <p>
-     * Note: Use with caution. By default retries are disabled.
-     * </p>
-     * @return this AtomicCounterBuilder
-     */
-    AtomicCounterBuilder withRetryOnFailure();
-
-    /**
      * Instantiates Metering service to gather usage and performance metrics.
      * By default, usage data will be stored.
      *
@@ -67,16 +56,6 @@
     AtomicCounterBuilder withMeteringDisabled();
 
     /**
-     * Sets the executor service to use for retrying failed operations.
-     * <p>
-     * Note: Must be set when retries are enabled
-     * </p>
-     * @param executor executor service
-     * @return this AtomicCounterBuilder
-     */
-    AtomicCounterBuilder withRetryExecutor(ScheduledExecutorService executor);
-
-    /**
      * Builds a AtomicCounter based on the configuration options
      * supplied to this builder.
      *
diff --git a/core/api/src/test/java/org/onosproject/store/service/TestAtomicCounter.java b/core/api/src/test/java/org/onosproject/store/service/TestAtomicCounter.java
index 49db644..01209be 100644
--- a/core/api/src/test/java/org/onosproject/store/service/TestAtomicCounter.java
+++ b/core/api/src/test/java/org/onosproject/store/service/TestAtomicCounter.java
@@ -15,7 +15,6 @@
  */
 package org.onosproject.store.service;
 
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -69,21 +68,11 @@
         }
 
         @Override
-        public AtomicCounterBuilder withRetryOnFailure() {
-            return this;
-        }
-
-        @Override
         public AtomicCounterBuilder withMeteringDisabled() {
             return this;
         }
 
         @Override
-        public AtomicCounterBuilder withRetryExecutor(ScheduledExecutorService executor) {
-            return this;
-        }
-
-        @Override
         public AsyncAtomicCounter buildAsyncCounter() {
             throw new UnsupportedOperationException("Async Counter is not supported");
         }
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAsyncAtomicCounter.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAsyncAtomicCounter.java
index 7ff6b89..8a9f884 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAsyncAtomicCounter.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAsyncAtomicCounter.java
@@ -16,15 +16,9 @@
 package org.onosproject.store.consistent.impl;
 
 import org.onosproject.store.service.AsyncAtomicCounter;
-import org.slf4j.Logger;
 
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.slf4j.LoggerFactory.getLogger;
 
 /**
  * Default implementation for a distributed AsyncAtomicCounter backed by
@@ -36,11 +30,6 @@
 
     private final String name;
     private final Database database;
-    private final boolean retryOnFailure;
-    private final ScheduledExecutorService retryExecutor;
-    // TODO: configure delay via builder
-    private static final int DELAY_BETWEEN_RETRY_SEC = 1;
-    private final Logger log = getLogger(getClass());
     private final MeteringAgent monitor;
 
     private static final String PRIMITIVE_NAME = "atomicCounter";
@@ -52,13 +41,9 @@
 
     public DefaultAsyncAtomicCounter(String name,
                                      Database database,
-                                     boolean retryOnException,
-                                     boolean meteringEnabled,
-                                     ScheduledExecutorService retryExecutor) {
+                                     boolean meteringEnabled) {
         this.name = checkNotNull(name);
         this.database = checkNotNull(database);
-        this.retryOnFailure = retryOnException;
-        this.retryExecutor = retryExecutor;
         this.monitor = new MeteringAgent(PRIMITIVE_NAME, name, meteringEnabled);
     }
 
@@ -86,77 +71,14 @@
     @Override
     public CompletableFuture<Long> getAndAdd(long delta) {
         final MeteringAgent.Context timer = monitor.startTimer(GET_AND_ADD);
-        CompletableFuture<Long> result = database.counterGetAndAdd(name, delta);
-        if (!retryOnFailure) {
-            return result
-                    .whenComplete((r, e) -> timer.stop());
-        }
-
-        CompletableFuture<Long> future = new CompletableFuture<>();
-        return result.whenComplete((r, e) -> {
-            timer.stop();
-            // TODO : Account for retries
-            if (e != null) {
-                log.warn("getAndAdd failed due to {}. Will retry", e.getMessage());
-                retryExecutor.schedule(new RetryTask(database::counterGetAndAdd, delta, future),
-                        DELAY_BETWEEN_RETRY_SEC,
-                        TimeUnit.SECONDS);
-            } else {
-                future.complete(r);
-            }
-        }).thenCompose(v -> future);
+        return database.counterGetAndAdd(name, delta)
+                       .whenComplete((r, e) -> timer.stop());
     }
 
     @Override
     public CompletableFuture<Long> addAndGet(long delta) {
         final MeteringAgent.Context timer = monitor.startTimer(ADD_AND_GET);
-        CompletableFuture<Long> result = database.counterAddAndGet(name, delta);
-        if (!retryOnFailure) {
-            return result
-                    .whenComplete((r, e) -> timer.stop());
-        }
-
-        CompletableFuture<Long> future = new CompletableFuture<>();
-        return result.whenComplete((r, e) -> {
-            timer.stop();
-            // TODO : Account for retries
-            if (e != null) {
-                log.warn("addAndGet failed due to {}. Will retry", e.getMessage());
-                retryExecutor.schedule(new RetryTask(database::counterAddAndGet, delta, future),
-                        DELAY_BETWEEN_RETRY_SEC,
-                        TimeUnit.SECONDS);
-            } else {
-                future.complete(r);
-            }
-        }).thenCompose(v -> future);
-    }
-
-    private class RetryTask implements Runnable {
-
-        private final BiFunction<String, Long, CompletableFuture<Long>> function;
-        private final Long delta;
-        private final CompletableFuture<Long> result;
-
-        public RetryTask(BiFunction<String, Long, CompletableFuture<Long>> function,
-                Long delta,
-                CompletableFuture<Long> result) {
-            this.function = function;
-            this.delta = delta;
-            this.result = result;
-        }
-
-        @Override
-        public void run() {
-            function.apply(name, delta).whenComplete((r, e) -> {
-                if (e == null) {
-                    result.complete(r);
-                } else {
-                    log.warn("{} retry failed due to {}. Will try again...", function, e.getMessage());
-                    // TODO: Exponential backoff
-                    // TODO: limit retries
-                    retryExecutor.schedule(this, DELAY_BETWEEN_RETRY_SEC, TimeUnit.SECONDS);
-                }
-            });
-        }
+        return database.counterAddAndGet(name, delta)
+                       .whenComplete((r, e) -> timer.stop());
     }
 }
\ No newline at end of file
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounter.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounter.java
index 483dbc4..64886e4 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounter.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounter.java
@@ -21,7 +21,6 @@
 
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -39,10 +38,8 @@
 
     public DefaultAtomicCounter(String name,
                                 Database database,
-                                boolean retryOnException,
-                                boolean meteringEnabled,
-                                ScheduledExecutorService retryExecutor) {
-        asyncCounter = new DefaultAsyncAtomicCounter(name, database, retryOnException, meteringEnabled, retryExecutor);
+                                boolean meteringEnabled) {
+        asyncCounter = new DefaultAsyncAtomicCounter(name, database, meteringEnabled);
     }
 
     @Override
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounterBuilder.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounterBuilder.java
index 4f0a296..dba4443 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounterBuilder.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DefaultAtomicCounterBuilder.java
@@ -19,9 +19,8 @@
 import org.onosproject.store.service.AtomicCounter;
 import org.onosproject.store.service.AtomicCounterBuilder;
 
-import java.util.concurrent.ScheduledExecutorService;
-
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
 
 /**
  * Default implementation of AtomicCounterBuilder.
@@ -32,9 +31,7 @@
     private boolean partitionsEnabled = true;
     private final Database partitionedDatabase;
     private final Database inMemoryDatabase;
-    private boolean retryOnFailure = false;
     private boolean metering = true;
-    private ScheduledExecutorService retryExecutor = null;
 
     public DefaultAtomicCounterBuilder(Database inMemoryDatabase, Database partitionedDatabase) {
         this.inMemoryDatabase = inMemoryDatabase;
@@ -58,20 +55,14 @@
     public AtomicCounter build() {
         validateInputs();
         Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase;
-        return new DefaultAtomicCounter(name, database, retryOnFailure, metering, retryExecutor);
+        return new DefaultAtomicCounter(name, database, metering);
     }
 
     @Override
     public AsyncAtomicCounter buildAsyncCounter() {
         validateInputs();
         Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase;
-        return new DefaultAsyncAtomicCounter(name, database, retryOnFailure, metering, retryExecutor);
-    }
-
-    @Override
-    public AtomicCounterBuilder withRetryOnFailure() {
-        retryOnFailure = true;
-        return this;
+        return new DefaultAsyncAtomicCounter(name, database, metering);
     }
 
     @Override
@@ -80,17 +71,7 @@
         return this;
     }
 
-    @Override
-    public AtomicCounterBuilder withRetryExecutor(ScheduledExecutorService executor) {
-        this.retryExecutor = executor;
-        return this;
-    }
-
     private void validateInputs() {
-        if (retryOnFailure) {
-            if (retryExecutor == null) {
-                throw new IllegalArgumentException("RetryExecutor must be specified when retries are enabled");
-            }
-        }
+        checkState(name != null, "name must be specified");
     }
 }
diff --git a/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentApplicationIdStore.java b/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentApplicationIdStore.java
index ecfc3dc..e54b0ee 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentApplicationIdStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentApplicationIdStore.java
@@ -15,12 +15,10 @@
  */
 package org.onosproject.store.core.impl;
 
-import static org.onlab.util.Tools.groupedThreads;
 import static org.slf4j.LoggerFactory.getLogger;
 
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 
 import org.apache.felix.scr.annotations.Activate;
@@ -30,20 +28,21 @@
 import org.apache.felix.scr.annotations.ReferenceCardinality;
 import org.apache.felix.scr.annotations.Service;
 import org.onlab.util.KryoNamespace;
+import org.onlab.util.Tools;
 import org.onosproject.core.ApplicationId;
 import org.onosproject.core.ApplicationIdStore;
 import org.onosproject.core.DefaultApplicationId;
 import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounter;
+import org.onosproject.store.service.AtomicCounter;
 import org.onosproject.store.service.ConsistentMap;
 import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.StorageException;
 import org.onosproject.store.service.StorageService;
 import org.onosproject.store.service.Versioned;
 import org.slf4j.Logger;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.Futures;
 
 /**
  * ApplicationIdStore implementation on top of {@code AtomicCounter}
@@ -58,7 +57,7 @@
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected StorageService storageService;
 
-    private AsyncAtomicCounter appIdCounter;
+    private AtomicCounter appIdCounter;
     private ConsistentMap<String, ApplicationId> registeredIds;
     private Map<String, ApplicationId> nameToAppIdCache = Maps.newConcurrentMap();
     private Map<Short, ApplicationId> idToAppIdCache = Maps.newConcurrentMap();
@@ -71,13 +70,10 @@
 
     @Activate
     public void activate() {
-        executor = Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/store/appId", "retry-handler"));
         appIdCounter = storageService.atomicCounterBuilder()
                                       .withName("onos-app-id-counter")
                                       .withPartitionsDisabled()
-                                      .withRetryOnFailure()
-                                      .withRetryExecutor(executor)
-                                      .buildAsyncCounter();
+                                      .build();
 
         registeredIds = storageService.<String, ApplicationId>consistentMapBuilder()
                 .withName("onos-app-ids")
@@ -128,7 +124,9 @@
         ApplicationId appId = nameToAppIdCache.computeIfAbsent(name, key -> {
             Versioned<ApplicationId> existingAppId = registeredIds.get(name);
             if (existingAppId == null) {
-                int id = Futures.getUnchecked(appIdCounter.incrementAndGet()).intValue();
+                int id = Tools.retryable(appIdCounter::incrementAndGet, StorageException.class, 1, 2000)
+                              .get()
+                              .intValue();
                 DefaultApplicationId newAppId = new DefaultApplicationId(id, name);
                 existingAppId = registeredIds.putIfAbsent(name, newAppId);
                 if (existingAppId != null) {
diff --git a/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentIdBlockStore.java b/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentIdBlockStore.java
index 8924423..8913742 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentIdBlockStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/core/impl/ConsistentIdBlockStore.java
@@ -1,12 +1,14 @@
 package org.onosproject.store.core.impl;
 
 import com.google.common.collect.Maps;
+
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
 import org.apache.felix.scr.annotations.Reference;
 import org.apache.felix.scr.annotations.ReferenceCardinality;
 import org.apache.felix.scr.annotations.Service;
+import org.onlab.util.Tools;
 import org.onosproject.core.IdBlock;
 import org.onosproject.core.IdBlockStore;
 import org.onosproject.store.service.AtomicCounter;
@@ -16,7 +18,6 @@
 
 import java.util.Map;
 
-import static org.onlab.util.Tools.randomDelay;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
@@ -54,19 +55,10 @@
                                  name -> storageService.atomicCounterBuilder()
                                          .withName(name)
                                          .build());
-        Throwable exc = null;
-        for (int i = 0; i < MAX_TRIES; i++) {
-            try {
-                Long blockBase = counter.getAndAdd(DEFAULT_BLOCK_SIZE);
-                return new IdBlock(blockBase, DEFAULT_BLOCK_SIZE);
-            } catch (StorageException e) {
-                log.warn("Unable to allocate ID block due to {}; retrying...",
-                         e.getMessage());
-                exc = e;
-                randomDelay(RETRY_DELAY_MS); // FIXME: This is a deliberate hack; fix in Drake
-            }
-        }
-        throw new IllegalStateException("Unable to allocate ID block", exc);
+        Long blockBase = Tools.retryable(counter::getAndAdd,
+                StorageException.class,
+                MAX_TRIES,
+                RETRY_DELAY_MS).apply(DEFAULT_BLOCK_SIZE);
+        return new IdBlock(blockBase, DEFAULT_BLOCK_SIZE);
     }
-
 }
diff --git a/incubator/store/src/main/java/org/onosproject/incubator/store/config/impl/DistributedNetworkConfigStore.java b/incubator/store/src/main/java/org/onosproject/incubator/store/config/impl/DistributedNetworkConfigStore.java
index 37d8a7f..606cfe2 100644
--- a/incubator/store/src/main/java/org/onosproject/incubator/store/config/impl/DistributedNetworkConfigStore.java
+++ b/incubator/store/src/main/java/org/onosproject/incubator/store/config/impl/DistributedNetworkConfigStore.java
@@ -26,6 +26,7 @@
 import com.fasterxml.jackson.databind.node.TextNode;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
+
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
@@ -165,14 +166,9 @@
 
     @Override
     public <S, T extends Config<S>> T getConfig(S subject, Class<T> configClass) {
-        // FIXME: There has to be a better way to absorb the timeout exceptions!
-        Versioned<ObjectNode> json = null;
-        try {
-            json = configs.get(key(subject, configClass));
-        } catch (ConsistentMapException e) {
-            Tools.randomDelay(MAX_BACKOFF);
-            json = configs.get(key(subject, configClass));
-        }
+        // TODO: need to identify and address the root cause for timeouts.
+        Versioned<ObjectNode> json = Tools.retryable(configs::get, ConsistentMapException.class, 1, MAX_BACKOFF)
+                                          .apply(key(subject, configClass));
         return json != null ? createConfig(subject, configClass, json.value()) : null;
     }
 
diff --git a/utils/misc/src/main/java/org/onlab/util/RetryingFunction.java b/utils/misc/src/main/java/org/onlab/util/RetryingFunction.java
new file mode 100644
index 0000000..484e236
--- /dev/null
+++ b/utils/misc/src/main/java/org/onlab/util/RetryingFunction.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2015 Open Networking Laboratory
+ *
+ * Licensed 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.onlab.util;
+
+import java.util.function.Function;
+
+import com.google.common.base.Throwables;
+
+/**
+ * Function that retries execution on failure.
+ *
+ * @param <U> input type
+ * @param <V> output type
+ */
+public class RetryingFunction<U, V> implements Function<U, V> {
+
+    private final Function<U, V> baseFunction;
+    private final Class<? extends Throwable> exceptionClass;
+    private final int maxRetries;
+    private final int maxDelayBetweenRetries;
+
+    public RetryingFunction(Function<U, V> baseFunction,
+            Class<? extends Throwable> exceptionClass,
+            int maxRetries,
+            int maxDelayBetweenRetries) {
+        this.baseFunction = baseFunction;
+        this.exceptionClass = exceptionClass;
+        this.maxRetries = maxRetries;
+        this.maxDelayBetweenRetries = maxDelayBetweenRetries;
+    }
+
+    @Override
+    public V apply(U input) {
+        int retryAttempts = 0;
+        while (true) {
+            try {
+                return baseFunction.apply(input);
+            } catch (Throwable t) {
+                if (!exceptionClass.isAssignableFrom(t.getClass()) || retryAttempts == maxRetries) {
+                    Throwables.propagate(t);
+                }
+                Tools.randomDelay(maxDelayBetweenRetries);
+                retryAttempts++;
+            }
+        }
+    }
+}
diff --git a/utils/misc/src/main/java/org/onlab/util/Tools.java b/utils/misc/src/main/java/org/onlab/util/Tools.java
index 301f411..b12d15c 100644
--- a/utils/misc/src/main/java/org/onlab/util/Tools.java
+++ b/utils/misc/src/main/java/org/onlab/util/Tools.java
@@ -46,6 +46,8 @@
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
@@ -227,6 +229,41 @@
     }
 
     /**
+     * Returns a function that retries execution on failure.
+     * @param base base function
+     * @param exceptionClass type of exception for which to retry
+     * @param maxRetries max number of retries before giving up
+     * @param maxDelayBetweenRetries max delay between successive retries. The actual delay is randomly picked from
+     * the interval (0, maxDelayBetweenRetries]
+     * @return function
+     */
+    public static <U, V> Function<U, V> retryable(Function<U, V> base,
+            Class<? extends Throwable> exceptionClass,
+            int maxRetries,
+            int maxDelayBetweenRetries) {
+        return new RetryingFunction<>(base, exceptionClass, maxRetries, maxDelayBetweenRetries);
+    }
+
+    /**
+     * Returns a Supplier that retries execution on failure.
+     * @param base base supplier
+     * @param exceptionClass type of exception for which to retry
+     * @param maxRetries max number of retries before giving up
+     * @param maxDelayBetweenRetries max delay between successive retries. The actual delay is randomly picked from
+     * the interval (0, maxDelayBetweenRetries]
+     * @return supplier
+     */
+    public static <V> Supplier<V> retryable(Supplier<V> base,
+            Class<? extends Throwable> exceptionClass,
+            int maxRetries,
+            int maxDelayBetweenRetries) {
+        return () -> new RetryingFunction<>(v -> base.get(),
+                exceptionClass,
+                maxRetries,
+                maxDelayBetweenRetries).apply(null);
+    }
+
+    /**
      * Suspends the current thread for a random number of millis between 0 and
      * the indicated limit.
      *
diff --git a/utils/misc/src/test/java/org/onlab/util/RetryingFunctionTest.java b/utils/misc/src/test/java/org/onlab/util/RetryingFunctionTest.java
new file mode 100644
index 0000000..4b08d2f
--- /dev/null
+++ b/utils/misc/src/test/java/org/onlab/util/RetryingFunctionTest.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2015 Open Networking Laboratory
+ *
+ * Licensed 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.onlab.util;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit tests for RetryingFunction.
+ *
+ */
+public class RetryingFunctionTest {
+
+    private int round;
+
+    @Before
+    public void setUp() {
+        round = 1;
+    }
+
+    @After
+    public void tearDown() {
+        round = 0;
+    }
+
+    @Test(expected = RetryableException.class)
+    public void testNoRetries() {
+        new RetryingFunction<>(this::succeedAfterOneFailure, RetryableException.class, 0, 10).apply(null);
+    }
+
+    @Test
+    public void testSuccessAfterOneRetry() {
+        new RetryingFunction<>(this::succeedAfterOneFailure, RetryableException.class, 1, 10).apply(null);
+    }
+
+    @Test(expected = RetryableException.class)
+    public void testFailureAfterOneRetry() {
+        new RetryingFunction<>(this::succeedAfterTwoFailures, RetryableException.class, 1, 10).apply(null);
+    }
+
+    @Test
+    public void testFailureAfterTwoRetries() {
+        new RetryingFunction<>(this::succeedAfterTwoFailures, RetryableException.class, 2, 10).apply(null);
+    }
+
+    @Test(expected = NonRetryableException.class)
+    public void testFailureWithNonRetryableFailure() {
+        new RetryingFunction<>(this::failCompletely, RetryableException.class, 2, 10).apply(null);
+    }
+
+    private String succeedAfterOneFailure(String input) {
+        if (round++ <= 1) {
+            throw new RetryableException();
+        } else {
+            return "pass";
+        }
+    }
+
+    private String succeedAfterTwoFailures(String input) {
+        if (round++ <= 2) {
+            throw new RetryableException();
+        } else {
+            return "pass";
+        }
+    }
+
+    private String failCompletely(String input) {
+        if (round++ <= 1) {
+            throw new NonRetryableException();
+        } else {
+            return "pass";
+        }
+    }
+
+    private class RetryableException extends RuntimeException {
+    }
+
+    private class NonRetryableException extends RuntimeException {
+    }
+}