Support for retries in AtomicCounter. Enabled counter operation retries in ConsistentApplicationIdStore

Change-Id: I705c51b2efd7ecd928c64c7f8a16d1965198253c
This commit is contained in:
Madan Jampani 2015-06-05 17:38:22 -07:00 committed by Gerrit Code Review
parent 20c452017b
commit f4d58f3ca5
5 changed files with 151 additions and 12 deletions

View File

@ -1,5 +1,7 @@
package org.onosproject.store.service; package org.onosproject.store.service;
import java.util.concurrent.ScheduledExecutorService;
/** /**
* Builder for AtomicCounter. * Builder for AtomicCounter.
*/ */
@ -32,6 +34,24 @@ public interface AtomicCounterBuilder {
*/ */
AtomicCounterBuilder withPartitionsDisabled(); AtomicCounterBuilder withPartitionsDisabled();
/**
* Enables retries when counter operations fail.
* <p>
* Note: Use with caution. By default retries are disabled.
* </p>
* @return this AtomicCounterBuilder
*/
AtomicCounterBuilder withRetryOnFailure();
/**
* Sets the executor service to use for retrying failed operations.
* <p>
* Note: Must be set when retries are enabled
* </p>
* @return this AtomicCounterBuilder
*/
AtomicCounterBuilder withRetryExecutor(ScheduledExecutorService executor);
/** /**
* Builds a AtomicCounter based on the configuration options * Builds a AtomicCounter based on the configuration options
* supplied to this builder. * supplied to this builder.

View File

@ -16,10 +16,20 @@
package org.onosproject.store.consistent.impl; package org.onosproject.store.consistent.impl;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import org.onosproject.store.service.AsyncAtomicCounter; import org.onosproject.store.service.AsyncAtomicCounter;
import org.slf4j.Logger;
import static com.google.common.base.Preconditions.*; import static com.google.common.base.Preconditions.*;
import static org.slf4j.LoggerFactory.getLogger;
/** /**
* Default implementation for a distributed AsyncAtomicCounter backed by * Default implementation for a distributed AsyncAtomicCounter backed by
@ -31,10 +41,20 @@ public class DefaultAsyncAtomicCounter implements AsyncAtomicCounter {
private final String name; private final String name;
private final Database database; 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());
public DefaultAsyncAtomicCounter(String name, Database database) { public DefaultAsyncAtomicCounter(String name,
Database database,
boolean retryOnException,
ScheduledExecutorService retryExecutor) {
this.name = checkNotNull(name); this.name = checkNotNull(name);
this.database = checkNotNull(database); this.database = checkNotNull(database);
this.retryOnFailure = retryOnException;
this.retryExecutor = retryExecutor;
} }
@Override @Override
@ -54,11 +74,70 @@ public class DefaultAsyncAtomicCounter implements AsyncAtomicCounter {
@Override @Override
public CompletableFuture<Long> getAndAdd(long delta) { public CompletableFuture<Long> getAndAdd(long delta) {
return database.counterGetAndAdd(name, delta); CompletableFuture<Long> result = database.counterGetAndAdd(name, delta);
if (!retryOnFailure) {
return result;
}
CompletableFuture<Long> future = new CompletableFuture<>();
return result.whenComplete((r, e) -> {
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);
} }
@Override @Override
public CompletableFuture<Long> addAndGet(long delta) { public CompletableFuture<Long> addAndGet(long delta) {
return database.counterAddAndGet(name, delta); CompletableFuture<Long> result = database.counterAddAndGet(name, delta);
if (!retryOnFailure) {
return result;
}
CompletableFuture<Long> future = new CompletableFuture<>();
return result.whenComplete((r, e) -> {
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);
}
});
}
} }
} }

View File

@ -17,6 +17,7 @@ package org.onosproject.store.consistent.impl;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
@ -36,8 +37,11 @@ public class DefaultAtomicCounter implements AtomicCounter {
private final AsyncAtomicCounter asyncCounter; private final AsyncAtomicCounter asyncCounter;
public DefaultAtomicCounter(String name, Database database) { public DefaultAtomicCounter(String name,
asyncCounter = new DefaultAsyncAtomicCounter(name, database); Database database,
boolean retryOnException,
ScheduledExecutorService retryExecutor) {
asyncCounter = new DefaultAsyncAtomicCounter(name, database, retryOnException, retryExecutor);
} }
@Override @Override

View File

@ -1,5 +1,7 @@
package org.onosproject.store.consistent.impl; package org.onosproject.store.consistent.impl;
import java.util.concurrent.ScheduledExecutorService;
import org.onosproject.store.service.AsyncAtomicCounter; import org.onosproject.store.service.AsyncAtomicCounter;
import org.onosproject.store.service.AtomicCounter; import org.onosproject.store.service.AtomicCounter;
import org.onosproject.store.service.AtomicCounterBuilder; import org.onosproject.store.service.AtomicCounterBuilder;
@ -15,6 +17,8 @@ public class DefaultAtomicCounterBuilder implements AtomicCounterBuilder {
private boolean partitionsEnabled = true; private boolean partitionsEnabled = true;
private final Database partitionedDatabase; private final Database partitionedDatabase;
private final Database inMemoryDatabase; private final Database inMemoryDatabase;
private boolean retryOnFailure = false;
private ScheduledExecutorService retryExecutor = null;
public DefaultAtomicCounterBuilder(Database inMemoryDatabase, Database partitionedDatabase) { public DefaultAtomicCounterBuilder(Database inMemoryDatabase, Database partitionedDatabase) {
this.inMemoryDatabase = inMemoryDatabase; this.inMemoryDatabase = inMemoryDatabase;
@ -36,13 +40,35 @@ public class DefaultAtomicCounterBuilder implements AtomicCounterBuilder {
@Override @Override
public AtomicCounter build() { public AtomicCounter build() {
validateInputs();
Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase; Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase;
return new DefaultAtomicCounter(name, database); return new DefaultAtomicCounter(name, database, retryOnFailure, retryExecutor);
} }
@Override @Override
public AsyncAtomicCounter buildAsyncCounter() { public AsyncAtomicCounter buildAsyncCounter() {
validateInputs();
Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase; Database database = partitionsEnabled ? partitionedDatabase : inMemoryDatabase;
return new DefaultAsyncAtomicCounter(name, database); return new DefaultAsyncAtomicCounter(name, database, retryOnFailure, retryExecutor);
}
@Override
public AtomicCounterBuilder withRetryOnFailure() {
retryOnFailure = true;
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");
}
}
} }
} }

View File

@ -15,10 +15,14 @@
*/ */
package org.onosproject.store.core.impl; package org.onosproject.store.core.impl;
import static org.onlab.util.Tools.groupedThreads;
import static org.slf4j.LoggerFactory.getLogger; import static org.slf4j.LoggerFactory.getLogger;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.felix.scr.annotations.Activate; import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Component; import org.apache.felix.scr.annotations.Component;
import org.apache.felix.scr.annotations.Deactivate; import org.apache.felix.scr.annotations.Deactivate;
@ -30,7 +34,7 @@ import org.onosproject.core.ApplicationId;
import org.onosproject.core.ApplicationIdStore; import org.onosproject.core.ApplicationIdStore;
import org.onosproject.core.DefaultApplicationId; import org.onosproject.core.DefaultApplicationId;
import org.onosproject.store.serializers.KryoNamespaces; import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.service.AtomicCounter; import org.onosproject.store.service.AsyncAtomicCounter;
import org.onosproject.store.service.ConsistentMap; import org.onosproject.store.service.ConsistentMap;
import org.onosproject.store.service.Serializer; import org.onosproject.store.service.Serializer;
import org.onosproject.store.service.StorageService; import org.onosproject.store.service.StorageService;
@ -39,6 +43,7 @@ import org.slf4j.Logger;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.util.concurrent.Futures;
/** /**
* ApplicationIdStore implementation on top of {@code AtomicCounter} * ApplicationIdStore implementation on top of {@code AtomicCounter}
@ -53,10 +58,11 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY) @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
protected StorageService storageService; protected StorageService storageService;
private AtomicCounter appIdCounter; private AsyncAtomicCounter appIdCounter;
private ConsistentMap<String, ApplicationId> registeredIds; private ConsistentMap<String, ApplicationId> registeredIds;
private Map<String, ApplicationId> nameToAppIdCache = Maps.newConcurrentMap(); private Map<String, ApplicationId> nameToAppIdCache = Maps.newConcurrentMap();
private Map<Short, ApplicationId> idToAppIdCache = Maps.newConcurrentMap(); private Map<Short, ApplicationId> idToAppIdCache = Maps.newConcurrentMap();
private ScheduledExecutorService executor;
private static final Serializer SERIALIZER = Serializer.using(new KryoNamespace.Builder() private static final Serializer SERIALIZER = Serializer.using(new KryoNamespace.Builder()
.register(KryoNamespaces.API) .register(KryoNamespaces.API)
@ -65,10 +71,13 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
@Activate @Activate
public void activate() { public void activate() {
executor = Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/store/appId", "retry-handler"));
appIdCounter = storageService.atomicCounterBuilder() appIdCounter = storageService.atomicCounterBuilder()
.withName("onos-app-id-counter") .withName("onos-app-id-counter")
.withPartitionsDisabled() .withPartitionsDisabled()
.build(); .withRetryOnFailure()
.withRetryExecutor(executor)
.buildAsyncCounter();
registeredIds = storageService.<String, ApplicationId>consistentMapBuilder() registeredIds = storageService.<String, ApplicationId>consistentMapBuilder()
.withName("onos-app-ids") .withName("onos-app-ids")
@ -83,6 +92,7 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
@Deactivate @Deactivate
public void deactivate() { public void deactivate() {
executor.shutdown();
log.info("Stopped"); log.info("Stopped");
} }
@ -118,7 +128,7 @@ public class ConsistentApplicationIdStore implements ApplicationIdStore {
ApplicationId appId = nameToAppIdCache.computeIfAbsent(name, key -> { ApplicationId appId = nameToAppIdCache.computeIfAbsent(name, key -> {
Versioned<ApplicationId> existingAppId = registeredIds.get(name); Versioned<ApplicationId> existingAppId = registeredIds.get(name);
if (existingAppId == null) { if (existingAppId == null) {
int id = (int) appIdCounter.incrementAndGet(); int id = Futures.getUnchecked(appIdCounter.incrementAndGet()).intValue();
DefaultApplicationId newAppId = new DefaultApplicationId(id, name); DefaultApplicationId newAppId = new DefaultApplicationId(id, name);
existingAppId = registeredIds.putIfAbsent(name, newAppId); existingAppId = registeredIds.putIfAbsent(name, newAppId);
if (existingAppId != null) { if (existingAppId != null) {