Madan Jampani
Committed by Gerrit Code Review

Added RetryingFunction for simplified retry support.

Moved retry logic out of primitives (AtomicCounter) to the caller site.

Change-Id: I319d61f153f98d421baf32a1b5cd69d20dc63427
......@@ -15,8 +15,6 @@
*/
package org.onosproject.store.service;
import java.util.concurrent.ScheduledExecutorService;
/**
* Builder for AtomicCounter.
*/
......@@ -50,15 +48,6 @@ public interface AtomicCounterBuilder {
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 @@ public interface AtomicCounterBuilder {
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.
*
......
......@@ -15,7 +15,6 @@
*/
package org.onosproject.store.service;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicLong;
/**
......@@ -69,21 +68,11 @@ public final class TestAtomicCounter implements AtomicCounter {
}
@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");
}
......
......@@ -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 @@ public class DefaultAsyncAtomicCounter implements AsyncAtomicCounter {
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 class DefaultAsyncAtomicCounter implements AsyncAtomicCounter {
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 @@ public class DefaultAsyncAtomicCounter implements AsyncAtomicCounter {
@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
......
......@@ -21,7 +21,6 @@ import org.onosproject.store.service.StorageException;
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 class DefaultAtomicCounter implements AtomicCounter {
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
......
......@@ -19,9 +19,8 @@ import org.onosproject.store.service.AsyncAtomicCounter;
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 @@ public class DefaultAtomicCounterBuilder implements AtomicCounterBuilder {
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 class DefaultAtomicCounterBuilder implements AtomicCounterBuilder {
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 @@ public class DefaultAtomicCounterBuilder implements AtomicCounterBuilder {
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");
}
}
......
......@@ -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.Reference;
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 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
@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 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
@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 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
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) {
......
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 org.slf4j.Logger;
import java.util.Map;
import static org.onlab.util.Tools.randomDelay;
import static org.slf4j.LoggerFactory.getLogger;
/**
......@@ -54,19 +55,10 @@ public class ConsistentIdBlockStore implements IdBlockStore {
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);
}
}
......
......@@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.node.ShortNode;
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 @@ public class DistributedNetworkConfigStore
@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;
}
......
/*
* 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++;
}
}
}
}
......@@ -46,6 +46,8 @@ import java.util.concurrent.Future;
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 @@ public abstract class Tools {
}
/**
* 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.
*
......
/*
* 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 {
}
}