X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShardCommitCoordinator.java;h=51d8d5caec18e0c94a22520fb23d1b9708acdbf4;hb=dd16edd5a758f0e51727de511f9868c72b2a1dd0;hp=6821fa721fe8b30629106f1d14ba666a775d9da6;hpb=14f8732b144cbe12c97a31555e794938cbe96e62;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardCommitCoordinator.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardCommitCoordinator.java index 6821fa721f..51d8d5caec 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardCommitCoordinator.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardCommitCoordinator.java @@ -8,19 +8,21 @@ package org.opendaylight.controller.cluster.datastore; import akka.actor.ActorRef; -import akka.actor.Status; +import akka.actor.Status.Failure; import akka.serialization.Serialization; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; +import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import org.opendaylight.controller.cluster.datastore.compat.BackwardsCompatibleThreePhaseCommitCohort; +import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply; import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply; @@ -28,8 +30,8 @@ import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTran import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction; import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply; import org.opendaylight.controller.cluster.datastore.modification.Modification; -import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification; import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException; +import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate; import org.slf4j.Logger; /** @@ -67,8 +69,10 @@ class ShardCommitCoordinator { private ReadyTransactionReply readyTransactionReply; + private Runnable runOnPendingTransactionsComplete; + ShardCommitCoordinator(ShardDataTree dataTree, - long cacheExpiryTimeoutInMillis, int queueCapacity, ActorRef shardActor, Logger log, String name) { + long cacheExpiryTimeoutInMillis, int queueCapacity, Logger log, String name) { this.queueCapacity = queueCapacity; this.log = log; @@ -77,6 +81,14 @@ class ShardCommitCoordinator { this.cacheExpiryTimeoutInMillis = cacheExpiryTimeoutInMillis; } + int getQueueSize() { + return queuedCohortEntries.size(); + } + + int getCohortCacheSize() { + return cohortCache.size(); + } + void setQueueCapacity(int queueCapacity) { this.queueCapacity = queueCapacity; } @@ -92,6 +104,10 @@ class ShardCommitCoordinator { private boolean queueCohortEntry(CohortEntry cohortEntry, ActorRef sender, Shard shard) { if(queuedCohortEntries.size() < queueCapacity) { queuedCohortEntries.offer(cohortEntry); + + log.debug("{}: Enqueued transaction {}, queue size {}", name, cohortEntry.getTransactionID(), + queuedCohortEntries.size()); + return true; } else { cohortCache.remove(cohortEntry.getTransactionID()); @@ -101,7 +117,7 @@ class ShardCommitCoordinator { " capacity %d has been reached.", name, cohortEntry.getTransactionID(), queueCapacity)); log.error(ex.getMessage()); - sender.tell(new Status.Failure(ex), shard.self()); + sender.tell(new Failure(ex), shard.self()); return false; } } @@ -118,42 +134,23 @@ class ShardCommitCoordinator { log.debug("{}: Readying transaction {}, client version {}", name, ready.getTransactionID(), ready.getTxnClientVersion()); - CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), ready.getCohort(), - (MutableCompositeModification) ready.getModification()); + ShardDataTreeCohort cohort = ready.getTransaction().ready(); + CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), cohort, ready.getTxnClientVersion()); cohortCache.put(ready.getTransactionID(), cohortEntry); if(!queueCohortEntry(cohortEntry, sender, shard)) { return; } - if(ready.getTxnClientVersion() < DataStoreVersions.LITHIUM_VERSION) { - // Return our actor path as we'll handle the three phase commit except if the Tx client - // version < Helium-1 version which means the Tx was initiated by a base Helium version node. - // In that case, the subsequent 3-phase commit messages won't contain the transactionId so to - // maintain backwards compatibility, we create a separate cohort actor to provide the compatible behavior. - ActorRef replyActorPath = shard.self(); - if(ready.getTxnClientVersion() < DataStoreVersions.HELIUM_1_VERSION) { - log.debug("{}: Creating BackwardsCompatibleThreePhaseCommitCohort", name); - replyActorPath = shard.getContext().actorOf(BackwardsCompatibleThreePhaseCommitCohort.props( - ready.getTransactionID())); - } - - ReadyTransactionReply readyTransactionReply = - new ReadyTransactionReply(Serialization.serializedActorPath(replyActorPath), - ready.getTxnClientVersion()); - sender.tell(ready.isReturnSerialized() ? readyTransactionReply.toSerializable() : - readyTransactionReply, shard.self()); + if(ready.isDoImmediateCommit()) { + cohortEntry.setDoImmediateCommit(true); + cohortEntry.setReplySender(sender); + cohortEntry.setShard(shard); + handleCanCommit(cohortEntry); } else { - if(ready.isDoImmediateCommit()) { - cohortEntry.setDoImmediateCommit(true); - cohortEntry.setReplySender(sender); - cohortEntry.setShard(shard); - handleCanCommit(cohortEntry); - } else { - // The caller does not want immediate commit - the 3-phase commit will be coordinated by the - // front-end so send back a ReadyTransactionReply with our actor path. - sender.tell(readyTransactionReply(shard), shard.self()); - } + // The caller does not want immediate commit - the 3-phase commit will be coordinated by the + // front-end so send back a ReadyTransactionReply with our actor path. + sender.tell(readyTransactionReply(shard), shard.self()); } } @@ -172,7 +169,7 @@ class ShardCommitCoordinator { if(cohortEntry == null) { cohortEntry = new CohortEntry(batched.getTransactionID(), dataTree.newReadWriteTransaction(batched.getTransactionID(), - batched.getTransactionChainID())); + batched.getTransactionChainID()), batched.getVersion()); cohortCache.put(batched.getTransactionID(), cohortEntry); } @@ -184,6 +181,18 @@ class ShardCommitCoordinator { cohortEntry.applyModifications(batched.getModifications()); if(batched.isReady()) { + if(cohortEntry.getLastBatchedModificationsException() != null) { + cohortCache.remove(cohortEntry.getTransactionID()); + throw cohortEntry.getLastBatchedModificationsException(); + } + + if(cohortEntry.getTotalBatchedModificationsReceived() != batched.getTotalMessagesSent()) { + cohortCache.remove(cohortEntry.getTransactionID()); + throw new IllegalStateException(String.format( + "The total number of batched messages received %d does not match the number sent %d", + cohortEntry.getTotalBatchedModificationsReceived(), batched.getTotalMessagesSent())); + } + if(!queueCohortEntry(cohortEntry, sender, shard)) { return; } @@ -218,7 +227,8 @@ class ShardCommitCoordinator { void handleReadyLocalTransaction(ReadyLocalTransaction message, ActorRef sender, Shard shard) { final ShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(dataTree, message.getModification(), message.getTransactionID()); - final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort); + final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort, + DataStoreVersions.CURRENT_VERSION); cohortCache.put(message.getTransactionID(), cohortEntry); cohortEntry.setDoImmediateCommit(message.isDoCommitOnReady()); @@ -285,7 +295,7 @@ class ShardCommitCoordinator { IllegalStateException ex = new IllegalStateException( String.format("%s: No cohort entry found for transaction %s", name, transactionID)); log.error(ex.getMessage()); - sender.tell(new Status.Failure(ex), shard.self()); + sender.tell(new Failure(ex), shard.self()); return; } @@ -298,10 +308,7 @@ class ShardCommitCoordinator { private void doCanCommit(final CohortEntry cohortEntry) { boolean canCommit = false; try { - // We block on the future here so we don't have to worry about possibly accessing our - // state on a different thread outside of our dispatcher. Also, the data store - // currently uses a same thread executor anyway. - canCommit = cohortEntry.getCohort().canCommit().get(); + canCommit = cohortEntry.canCommit(); log.debug("{}: canCommit for {}: {}", name, cohortEntry.getTransactionID(), canCommit); @@ -309,13 +316,15 @@ class ShardCommitCoordinator { if(canCommit) { doCommit(cohortEntry); } else { - cohortEntry.getReplySender().tell(new Status.Failure(new TransactionCommitFailedException( + cohortEntry.getReplySender().tell(new Failure(new TransactionCommitFailedException( "Can Commit failed, no detailed cause available.")), cohortEntry.getShard().self()); } } else { + // FIXME - use caller's version cohortEntry.getReplySender().tell( - canCommit ? CanCommitTransactionReply.YES.toSerializable() : - CanCommitTransactionReply.NO.toSerializable(), cohortEntry.getShard().self()); + canCommit ? CanCommitTransactionReply.yes(cohortEntry.getClientVersion()).toSerializable() : + CanCommitTransactionReply.no(cohortEntry.getClientVersion()).toSerializable(), + cohortEntry.getShard().self()); } } catch (Exception e) { log.debug("{}: An exception occurred during canCommit", name, e); @@ -325,7 +334,7 @@ class ShardCommitCoordinator { failure = e.getCause(); } - cohortEntry.getReplySender().tell(new Status.Failure(failure), cohortEntry.getShard().self()); + cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self()); } finally { if(!canCommit) { // Remove the entry from the cache now. @@ -345,10 +354,7 @@ class ShardCommitCoordinator { // normally fail since we ensure only one concurrent 3-phase commit. try { - // We block on the future here so we don't have to worry about possibly accessing our - // state on a different thread outside of our dispatcher. Also, the data store - // currently uses a same thread executor anyway. - cohortEntry.getCohort().preCommit().get(); + cohortEntry.preCommit(); cohortEntry.getShard().continueCommit(cohortEntry); @@ -358,7 +364,7 @@ class ShardCommitCoordinator { } catch (Exception e) { log.error("{} An exception occurred while preCommitting transaction {}", name, cohortEntry.getTransactionID(), e); - cohortEntry.getReplySender().tell(new akka.actor.Status.Failure(e), cohortEntry.getShard().self()); + cohortEntry.getReplySender().tell(new Failure(e), cohortEntry.getShard().self()); currentTransactionComplete(cohortEntry.getTransactionID(), true); } @@ -385,7 +391,7 @@ class ShardCommitCoordinator { String.format("%s: Cannot commit transaction %s - it is not the current transaction", name, transactionID)); log.error(ex.getMessage()); - sender.tell(new akka.actor.Status.Failure(ex), shard.self()); + sender.tell(new Failure(ex), shard.self()); return false; } @@ -393,6 +399,77 @@ class ShardCommitCoordinator { return doCommit(cohortEntry); } + void handleAbort(final String transactionID, final ActorRef sender, final Shard shard) { + CohortEntry cohortEntry = getCohortEntryIfCurrent(transactionID); + if(cohortEntry != null) { + // We don't remove the cached cohort entry here (ie pass false) in case the Tx was + // aborted during replication in which case we may still commit locally if replication + // succeeds. + currentTransactionComplete(transactionID, false); + } else { + cohortEntry = getAndRemoveCohortEntry(transactionID); + } + + if(cohortEntry == null) { + return; + } + + log.debug("{}: Aborting transaction {}", name, transactionID); + + final ActorRef self = shard.getSelf(); + try { + cohortEntry.abort(); + + shard.getShardMBean().incrementAbortTransactionsCount(); + + if(sender != null) { + sender.tell(AbortTransactionReply.instance(cohortEntry.getClientVersion()).toSerializable(), self); + } + } catch (Exception e) { + log.error("{}: An exception happened during abort", name, e); + + if(sender != null) { + sender.tell(new Failure(e), self); + } + } + } + + void checkForExpiredTransactions(final long timeout, final Shard shard) { + CohortEntry cohortEntry = getCurrentCohortEntry(); + if(cohortEntry != null) { + if(cohortEntry.isExpired(timeout)) { + log.warn("{}: Current transaction {} has timed out after {} ms - aborting", + name, cohortEntry.getTransactionID(), timeout); + + handleAbort(cohortEntry.getTransactionID(), null, shard); + } + } + + cleanupExpiredCohortEntries(); + } + + void abortPendingTransactions(final String reason, final Shard shard) { + if(currentCohortEntry == null && queuedCohortEntries.isEmpty()) { + return; + } + + List cohortEntries = new ArrayList<>(); + + if(currentCohortEntry != null) { + cohortEntries.add(currentCohortEntry); + currentCohortEntry = null; + } + + cohortEntries.addAll(queuedCohortEntries); + queuedCohortEntries.clear(); + + for(CohortEntry cohortEntry: cohortEntries) { + if(cohortEntry.getReplySender() != null) { + cohortEntry.getReplySender().tell(new Failure(new RuntimeException(reason)), shard.self()); + } + } + } + /** * Returns the cohort entry for the Tx commit currently in progress if the given transaction ID * matches the current entry. @@ -401,7 +478,7 @@ class ShardCommitCoordinator { * @return the current CohortEntry or null if the given transaction ID does not match the * current entry. */ - public CohortEntry getCohortEntryIfCurrent(String transactionID) { + CohortEntry getCohortEntryIfCurrent(String transactionID) { if(isCurrentTransaction(transactionID)) { return currentCohortEntry; } @@ -409,15 +486,15 @@ class ShardCommitCoordinator { return null; } - public CohortEntry getCurrentCohortEntry() { + CohortEntry getCurrentCohortEntry() { return currentCohortEntry; } - public CohortEntry getAndRemoveCohortEntry(String transactionID) { + CohortEntry getAndRemoveCohortEntry(String transactionID) { return cohortCache.remove(transactionID); } - public boolean isCurrentTransaction(String transactionID) { + boolean isCurrentTransaction(String transactionID) { return currentCohortEntry != null && currentCohortEntry.getTransactionID().equals(transactionID); } @@ -431,7 +508,7 @@ class ShardCommitCoordinator { * @param removeCohortEntry if true the CohortEntry for the transaction is also removed from * the cache. */ - public void currentTransactionComplete(String transactionID, boolean removeCohortEntry) { + void currentTransactionComplete(String transactionID, boolean removeCohortEntry) { if(removeCohortEntry) { cohortCache.remove(transactionID); } @@ -467,19 +544,35 @@ class ShardCommitCoordinator { } else if(next.isExpired(cacheExpiryTimeoutInMillis)) { log.warn("{}: canCommit for transaction {} was not received within {} ms - entry removed from cache", name, next.getTransactionID(), cacheExpiryTimeoutInMillis); - - iter.remove(); - cohortCache.remove(next.getTransactionID()); - } else { + } else if(!next.isAborted()) { break; } + + iter.remove(); + cohortCache.remove(next.getTransactionID()); } + + maybeRunOperationOnPendingTransactionsComplete(); } void cleanupExpiredCohortEntries() { maybeProcessNextCohortEntry(); } + void setRunOnPendingTransactionsComplete(Runnable operation) { + runOnPendingTransactionsComplete = operation; + maybeRunOperationOnPendingTransactionsComplete(); + } + + private void maybeRunOperationOnPendingTransactionsComplete() { + if(runOnPendingTransactionsComplete != null && currentCohortEntry == null && queuedCohortEntries.isEmpty()) { + log.debug("{}: Pending transactions complete - running operation {}", name, runOnPendingTransactionsComplete); + + runOnPendingTransactionsComplete.run(); + runOnPendingTransactionsComplete = null; + } + } + @VisibleForTesting void setCohortDecorator(CohortDecorator cohortDecorator) { this.cohortDecorator = cohortDecorator; @@ -489,27 +582,26 @@ class ShardCommitCoordinator { private final String transactionID; private ShardDataTreeCohort cohort; private final ReadWriteShardDataTreeTransaction transaction; + private RuntimeException lastBatchedModificationsException; private ActorRef replySender; private Shard shard; private boolean doImmediateCommit; private final Stopwatch lastAccessTimer = Stopwatch.createStarted(); + private int totalBatchedModificationsReceived; + private boolean aborted; + private final short clientVersion; - CohortEntry(String transactionID, ReadWriteShardDataTreeTransaction transaction) { + CohortEntry(String transactionID, ReadWriteShardDataTreeTransaction transaction, short clientVersion) { this.transaction = Preconditions.checkNotNull(transaction); this.transactionID = transactionID; + this.clientVersion = clientVersion; } - CohortEntry(String transactionID, ShardDataTreeCohort cohort, - MutableCompositeModification compositeModification) { - this.transactionID = transactionID; - this.cohort = cohort; - this.transaction = null; - } - - CohortEntry(String transactionID, ShardDataTreeCohort cohort) { + CohortEntry(String transactionID, ShardDataTreeCohort cohort, short clientVersion) { this.transactionID = transactionID; this.cohort = cohort; this.transaction = null; + this.clientVersion = clientVersion; } void updateLastAccessTime() { @@ -521,16 +613,58 @@ class ShardCommitCoordinator { return transactionID; } - ShardDataTreeCohort getCohort() { - return cohort; + short getClientVersion() { + return clientVersion; + } + + DataTreeCandidate getCandidate() { + return cohort.getCandidate(); + } + + int getTotalBatchedModificationsReceived() { + return totalBatchedModificationsReceived; + } + + RuntimeException getLastBatchedModificationsException() { + return lastBatchedModificationsException; } void applyModifications(Iterable modifications) { - for (Modification modification : modifications) { - modification.apply(transaction.getSnapshot()); + totalBatchedModificationsReceived++; + if(lastBatchedModificationsException == null) { + for (Modification modification : modifications) { + try { + modification.apply(transaction.getSnapshot()); + } catch (RuntimeException e) { + lastBatchedModificationsException = e; + throw e; + } + } } } + boolean canCommit() throws InterruptedException, ExecutionException { + // We block on the future here (and also preCommit(), commit(), abort()) so we don't have to worry + // about possibly accessing our state on a different thread outside of our dispatcher. + // TODO: the ShardDataTreeCohort returns immediate Futures anyway which begs the question - why + // bother even returning Futures from ShardDataTreeCohort if we have to treat them synchronously + // anyway?. The Futures are really a remnant from when we were using the InMemoryDataBroker. + return cohort.canCommit().get(); + } + + void preCommit() throws InterruptedException, ExecutionException { + cohort.preCommit().get(); + } + + void commit() throws InterruptedException, ExecutionException { + cohort.commit().get(); + } + + void abort() throws InterruptedException, ExecutionException { + aborted = true; + cohort.abort().get(); + } + void ready(CohortDecorator cohortDecorator, boolean doImmediateCommit) { Preconditions.checkState(cohort == null, "cohort was already set"); @@ -576,6 +710,11 @@ class ShardCommitCoordinator { this.shard = shard; } + + boolean isAborted() { + return aborted; + } + @Override public String toString() { StringBuilder builder = new StringBuilder();