import akka.actor.ActorSelection;
import akka.dispatch.OnComplete;
import com.google.common.base.Preconditions;
-import java.util.AbstractMap.SimpleEntry;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
import org.opendaylight.controller.md.sal.common.api.data.TransactionChainClosedException;
private interface State {
boolean isReady();
- SimpleEntry<Object, List<Future<ActorSelection>>> getReadyFutures();
-
- void setReadyFutures(Object txIdentifier, List<Future<ActorSelection>> readyFutures);
+ List<Future<ActorSelection>> getPreviousReadyFutures();
}
private static class Allocated implements State {
- private volatile SimpleEntry<Object, List<Future<ActorSelection>>> readyFutures;
+ private final ChainedTransactionProxy transaction;
- @Override
- public boolean isReady() {
- return readyFutures != null;
+ Allocated(ChainedTransactionProxy transaction) {
+ this.transaction = transaction;
}
@Override
- public SimpleEntry<Object, List<Future<ActorSelection>>> getReadyFutures() {
- return readyFutures != null ? readyFutures : EMPTY_READY_FUTURES;
+ public boolean isReady() {
+ return transaction.isReady();
}
@Override
- public void setReadyFutures(Object txIdentifier, List<Future<ActorSelection>> readyFutures) {
- this.readyFutures = new SimpleEntry<>(txIdentifier, readyFutures);
+ public List<Future<ActorSelection>> getPreviousReadyFutures() {
+ return transaction.getReadyFutures();
}
}
private static abstract class AbstractDefaultState implements State {
@Override
- public SimpleEntry<Object, List<Future<ActorSelection>>> getReadyFutures() {
- return EMPTY_READY_FUTURES;
- }
-
- @Override
- public void setReadyFutures(Object txIdentifier, List<Future<ActorSelection>> readyFutures) {
- throw new IllegalStateException("No transaction is allocated");
+ public List<Future<ActorSelection>> getPreviousReadyFutures() {
+ return Collections.emptyList();
}
}
}
};
- private static final SimpleEntry<Object, List<Future<ActorSelection>>> EMPTY_READY_FUTURES =
- new SimpleEntry<Object, List<Future<ActorSelection>>>("",
- Collections.<Future<ActorSelection>>emptyList());
-
- private static final AtomicReferenceFieldUpdater<TransactionChainProxy, State> STATE_UPDATER =
- AtomicReferenceFieldUpdater.newUpdater(TransactionChainProxy.class, State.class, "state");
+ private static final AtomicInteger counter = new AtomicInteger(0);
private final ActorContext actorContext;
private final String transactionChainId;
- private volatile State state = IDLE_STATE;
- private static final AtomicInteger counter = new AtomicInteger(0);
+ private volatile State currentState = IDLE_STATE;
public TransactionChainProxy(ActorContext actorContext) {
this.actorContext = actorContext;
- transactionChainId = actorContext.getCurrentMemberName() + "-transaction-chain-" + counter.incrementAndGet();
+ transactionChainId = actorContext.getCurrentMemberName() + "-txn-chain-" + counter.incrementAndGet();
}
public String getTransactionChainId() {
@Override
public DOMStoreReadTransaction newReadOnlyTransaction() {
- checkReadyState();
- return new ChainedTransactionProxy(actorContext, TransactionProxy.TransactionType.READ_ONLY);
+ State localState = currentState;
+ checkReadyState(localState);
+
+ return new ChainedTransactionProxy(actorContext, TransactionProxy.TransactionType.READ_ONLY,
+ transactionChainId, localState.getPreviousReadyFutures());
}
@Override
@Override
public void close() {
- state = CLOSED_STATE;
+ currentState = CLOSED_STATE;
// Send a close transaction chain request to each and every shard
actorContext.broadcast(new CloseTransactionChain(transactionChainId));
}
private ChainedTransactionProxy allocateWriteTransaction(TransactionProxy.TransactionType type) {
- checkReadyState();
+ State localState = currentState;
+
+ checkReadyState(localState);
- ChainedTransactionProxy txProxy = new ChainedTransactionProxy(actorContext, type);
- STATE_UPDATER.compareAndSet(this, IDLE_STATE, new Allocated());
+ // Pass the ready Futures from the previous Tx.
+ ChainedTransactionProxy txProxy = new ChainedTransactionProxy(actorContext, type,
+ transactionChainId, localState.getPreviousReadyFutures());
+
+ currentState = new Allocated(txProxy);
return txProxy;
}
- private void checkReadyState() {
- Preconditions.checkState(state.isReady(), "Previous transaction %s is not ready yet",
- state.getReadyFutures().getKey());
+ private void checkReadyState(State state) {
+ Preconditions.checkState(state.isReady(), "Previous transaction is not ready yet");
}
- private class ChainedTransactionProxy extends TransactionProxy {
+ private static class ChainedTransactionProxy extends TransactionProxy {
+
+ /**
+ * Stores the ready Futures from the previous Tx in the chain.
+ */
+ private final List<Future<ActorSelection>> previousReadyFutures;
+
+ /**
+ * Stores the ready Futures from this transaction when it is readied.
+ */
+ private volatile List<Future<ActorSelection>> readyFutures;
- ChainedTransactionProxy(ActorContext actorContext, TransactionType transactionType) {
+ private ChainedTransactionProxy(ActorContext actorContext, TransactionType transactionType,
+ String transactionChainId, List<Future<ActorSelection>> previousReadyFutures) {
super(actorContext, transactionType, transactionChainId);
+ this.previousReadyFutures = previousReadyFutures;
+ }
+
+ List<Future<ActorSelection>> getReadyFutures() {
+ return readyFutures;
+ }
+
+ boolean isReady() {
+ return readyFutures != null;
}
@Override
protected void onTransactionReady(List<Future<ActorSelection>> readyFutures) {
- LOG.debug("onTransactionReady {} pending readyFutures size {} chain {}", getIdentifier(), readyFutures.size(), TransactionChainProxy.this.transactionChainId);
- state.setReadyFutures(getIdentifier(), readyFutures);
+ LOG.debug("onTransactionReady {} pending readyFutures size {} chain {}", getIdentifier(),
+ readyFutures.size(), getTransactionChainId());
+ this.readyFutures = readyFutures;
}
/**
final Object serializedCreateMessage) {
// Check if there are any previous ready Futures, otherwise let the super class handle it.
- // The second check is done to ensure the the previous ready Futures aren't for this
- // Tx instance as deadlock would occur if we tried to wait on our own Futures. This can
- // occur in this scenario:
- //
- // - the TransactionProxy is created and the client does a write.
- //
- // - the TransactionProxy then attempts to create the shard Tx. However it first
- // sends a FindPrimaryShard message to the shard manager to find the local shard
- // This call is done async.
- //
- // - the client submits the Tx and the TransactionProxy is readied and we cache
- // the ready Futures here.
- //
- // - then the FindPrimaryShard call completes and this method is called to create
- // the shard Tx. However the cached Futures were from the ready on this Tx. If we
- // tried to wait on them, it would cause a form of deadlock as the ready Future
- // would be waiting on the Tx create Future and vice versa.
- SimpleEntry<Object, List<Future<ActorSelection>>> readyFuturesEntry = state.getReadyFutures();
- List<Future<ActorSelection>> readyFutures = readyFuturesEntry.getValue();
- if(readyFutures.isEmpty() || getIdentifier().equals(readyFuturesEntry.getKey())) {
+ if(previousReadyFutures.isEmpty()) {
return super.sendCreateTransaction(shard, serializedCreateMessage);
}
// Combine the ready Futures into 1.
Future<Iterable<ActorSelection>> combinedFutures = akka.dispatch.Futures.sequence(
- readyFutures, actorContext.getActorSystem().dispatcher());
+ previousReadyFutures, getActorContext().getActorSystem().dispatcher());
// Add a callback for completion of the combined Futures.
final Promise<Object> createTxPromise = akka.dispatch.Futures.promise();
// A Ready Future failed so fail the returned Promise.
createTxPromise.failure(failure);
} else {
+ LOG.debug("Previous Tx readied - sending CreateTransaction for {} on chain {}",
+ getIdentifier(), getTransactionChainId());
+
// Send the CreateTx message and use the resulting Future to complete the
// returned Promise.
- createTxPromise.completeWith(actorContext.executeOperationAsync(shard,
+ createTxPromise.completeWith(getActorContext().executeOperationAsync(shard,
serializedCreateMessage));
}
}
};
- combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+ combinedFutures.onComplete(onComplete, getActorContext().getActorSystem().dispatcher());
return createTxPromise.future();
}
import com.google.common.base.Optional;
import com.google.common.util.concurrent.CheckedFuture;
import com.google.common.util.concurrent.Uninterruptibles;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
}};
}
+ @Test
+ public void testCreateChainedTransactionsInQuickSuccession() throws Exception{
+ new IntegrationTestKit(getSystem()) {{
+ DistributedDataStore dataStore = setupDistributedDataStore(
+ "testCreateChainedTransactionsInQuickSuccession", "test-1");
+
+ DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+
+ NormalizedNode<?, ?> testNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+
+ int nTxs = 20;
+ List<DOMStoreThreePhaseCommitCohort> cohorts = new ArrayList<>(nTxs);
+ for(int i = 0; i < nTxs; i++) {
+ DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
+
+ rwTx.merge(TestModel.TEST_PATH, testNode);
+
+ cohorts.add(rwTx.ready());
+
+ }
+
+ for(DOMStoreThreePhaseCommitCohort cohort: cohorts) {
+ doCommit(cohort);
+ }
+
+ txChain.close();
+
+ cleanup(dataStore);
+ }};
+ }
+
+ @Test
+ public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception{
+ new IntegrationTestKit(getSystem()) {{
+ DistributedDataStore dataStore = setupDistributedDataStore(
+ "testCreateChainedTransactionAfterEmptyTxReadied", "test-1");
+
+ DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
+
+ DOMStoreReadWriteTransaction rwTx1 = txChain.newReadWriteTransaction();
+
+ rwTx1.ready();
+
+ DOMStoreReadWriteTransaction rwTx2 = txChain.newReadWriteTransaction();
+
+ Optional<NormalizedNode<?, ?>> optional = rwTx2.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
+ assertEquals("isPresent", false, optional.isPresent());
+
+ txChain.close();
+
+ cleanup(dataStore);
+ }};
+ }
+
@Test
public void testCreateChainedTransactionWhenPreviousNotReady() throws Throwable {
new IntegrationTestKit(getSystem()) {{