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%2FTransactionProxy.java;h=b1b0a862ff4e48eaf29639e7181a8e10a50a7276;hb=refs%2Fchanges%2F57%2F72657%2F3;hp=c3a5d074140785626828c0d6eebaf18709ed5bbc;hpb=380b9ee20a2032cd223e66dded8204519540803f;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java index c3a5d07414..b1b0a862ff 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java @@ -12,26 +12,34 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.CheckedFuture; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier; -import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory; +import java.util.TreeMap; +import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier; +import org.opendaylight.controller.cluster.datastore.messages.AbstractRead; +import org.opendaylight.controller.cluster.datastore.messages.DataExists; +import org.opendaylight.controller.cluster.datastore.messages.ReadData; +import org.opendaylight.controller.cluster.datastore.modification.AbstractModification; +import org.opendaylight.controller.cluster.datastore.modification.DeleteModification; +import org.opendaylight.controller.cluster.datastore.modification.MergeModification; +import org.opendaylight.controller.cluster.datastore.modification.WriteModification; import org.opendaylight.controller.cluster.datastore.utils.ActorContext; import org.opendaylight.controller.cluster.datastore.utils.NormalizedNodeAggregator; import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException; import org.opendaylight.controller.sal.core.spi.data.AbstractDOMStoreTransaction; import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction; -import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture; +import org.opendaylight.mdsal.common.api.MappingCheckedFuture; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode; import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; @@ -43,15 +51,39 @@ import scala.concurrent.Promise; /** * A transaction potentially spanning multiple backend shards. */ -public class TransactionProxy extends AbstractDOMStoreTransaction implements DOMStoreReadWriteTransaction { - private static enum TransactionState { +public class TransactionProxy extends AbstractDOMStoreTransaction + implements DOMStoreReadWriteTransaction { + private enum TransactionState { OPEN, READY, CLOSED, } + private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class); - private final Map txContextWrappers = new HashMap<>(); + // Global lock used for transactions spanning multiple shards - synchronizes sending of the ready messages + // for atomicity to avoid potential deadlock with concurrent transactions spanning the same shards as outlined + // in the following scenario: + // + // - Tx1 sends ready message to shard A + // - Tx2 sends ready message to shard A + // - Tx2 sends ready message to shard B + // - Tx1 sends ready message to shard B + // + // This scenario results in deadlock: after Tx1 canCommits to shard A, it can't proceed with shard B until Tx2 + // completes as Tx2 was readied first on shard B. However Tx2 cannot make progress because it's waiting to canCommit + // on shard A which is blocked by Tx1. + // + // The global lock avoids this as it forces the ready messages to be sent in a predictable order: + // + // - Tx1 sends ready message to shard A + // - Tx1 sends ready message to shard B + // - Tx2 sends ready message to shard A + // - Tx2 sends ready message to shard B + // + private static final Object GLOBAL_TX_READY_LOCK = new Object(); + + private final Map txContextWrappers = new TreeMap<>(); private final AbstractTransactionContextFactory txContextFactory; private final TransactionType type; private TransactionState state = TransactionState.OPEN; @@ -68,16 +100,22 @@ public class TransactionProxy extends AbstractDOMStoreTransaction exists(final YangInstanceIdentifier path) { - Preconditions.checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); + return executeRead(shardNameFromIdentifier(path), new DataExists(path, DataStoreVersions.CURRENT_VERSION)); + } + + private CheckedFuture executeRead(final String shardName, + final AbstractRead readCmd) { + Preconditions.checkState(type != TransactionType.WRITE_ONLY, + "Reads from write-only transactions are not allowed"); - LOG.debug("Tx {} exists {}", getIdentifier(), path); + LOG.trace("Tx {} {} {}", getIdentifier(), readCmd.getClass().getSimpleName(), readCmd.getPath()); - final SettableFuture proxyFuture = SettableFuture.create(); - TransactionContextWrapper contextWrapper = getContextWrapper(path); + final SettableFuture proxyFuture = SettableFuture.create(); + TransactionContextWrapper contextWrapper = getContextWrapper(shardName); contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.dataExists(path, proxyFuture); + public void invoke(final TransactionContext transactionContext, final Boolean havePermit) { + transactionContext.executeRead(readCmd, proxyFuture, havePermit); } }); @@ -86,9 +124,10 @@ public class TransactionProxy extends AbstractDOMStoreTransaction>, ReadFailedException> read(final YangInstanceIdentifier path) { - Preconditions.checkState(type != TransactionType.WRITE_ONLY, "Reads from write-only transactions are not allowed"); + Preconditions.checkState(type != TransactionType.WRITE_ONLY, + "Reads from write-only transactions are not allowed"); - LOG.debug("Tx {} read {}", getIdentifier(), path); + LOG.trace("Tx {} read {}", getIdentifier(), path); if (YangInstanceIdentifier.EMPTY.equals(path)) { return readAllData(); @@ -99,21 +138,13 @@ public class TransactionProxy extends AbstractDOMStoreTransaction>, ReadFailedException> singleShardRead( final String shardName, final YangInstanceIdentifier path) { - final SettableFuture>> proxyFuture = SettableFuture.create(); - TransactionContextWrapper contextWrapper = getContextWrapper(shardName); - contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.readData(path, proxyFuture); - } - }); - - return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER); + return executeRead(shardName, new ReadData(path, DataStoreVersions.CURRENT_VERSION)); } private CheckedFuture>, ReadFailedException> readAllData() { final Set allShardNames = txContextFactory.getActorContext().getConfiguration().getAllShardNames(); - final Collection>, ReadFailedException>> futures = new ArrayList<>(allShardNames.size()); + final Collection>, ReadFailedException>> futures = + new ArrayList<>(allShardNames.size()); for (String shardName : allShardNames) { futures.add(singleShardRead(shardName, YangInstanceIdentifier.EMPTY)); @@ -122,61 +153,46 @@ public class TransactionProxy extends AbstractDOMStoreTransaction>>> listFuture = Futures.allAsList(futures); final ListenableFuture>> aggregateFuture; - aggregateFuture = Futures.transform(listFuture, new Function>>, Optional>>() { - @Override - public Optional> apply(final List>> input) { + aggregateFuture = Futures.transform(listFuture, + (Function>>, Optional>>) input -> { try { - return NormalizedNodeAggregator.aggregate(YangInstanceIdentifier.EMPTY, input, txContextFactory.getActorContext().getSchemaContext()); + return NormalizedNodeAggregator.aggregate(YangInstanceIdentifier.EMPTY, input, + txContextFactory.getActorContext().getSchemaContext(), + txContextFactory.getActorContext().getDatastoreContext().getLogicalStoreType()); } catch (DataValidationFailedException e) { throw new IllegalArgumentException("Failed to aggregate", e); } - } - }); + }, MoreExecutors.directExecutor()); return MappingCheckedFuture.create(aggregateFuture, ReadFailedException.MAPPER); } @Override public void delete(final YangInstanceIdentifier path) { - checkModificationState(); - - LOG.debug("Tx {} delete {}", getIdentifier(), path); - - TransactionContextWrapper contextWrapper = getContextWrapper(path); - contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.deleteData(path); - } - }); + executeModification(new DeleteModification(path)); } @Override public void merge(final YangInstanceIdentifier path, final NormalizedNode data) { - checkModificationState(); - - LOG.debug("Tx {} merge {}", getIdentifier(), path); - - TransactionContextWrapper contextWrapper = getContextWrapper(path); - contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { - @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.mergeData(path, data); - } - }); + executeModification(new MergeModification(path, data)); } @Override public void write(final YangInstanceIdentifier path, final NormalizedNode data) { + executeModification(new WriteModification(path, data)); + } + + private void executeModification(final AbstractModification modification) { checkModificationState(); - LOG.debug("Tx {} write {}", getIdentifier(), path); + LOG.trace("Tx {} executeModification {} {}", getIdentifier(), modification.getClass().getSimpleName(), + modification.getPath()); - TransactionContextWrapper contextWrapper = getContextWrapper(path); + TransactionContextWrapper contextWrapper = getContextWrapper(modification.getPath()); contextWrapper.maybeExecuteTransactionOperation(new TransactionOperation() { @Override - public void invoke(TransactionContext transactionContext) { - transactionContext.writeData(path, data); + protected void invoke(final TransactionContext transactionContext, final Boolean havePermit) { + transactionContext.executeModification(modification, havePermit); } }); } @@ -209,7 +225,7 @@ public class TransactionProxy extends AbstractDOMStoreTransaction ret; switch (txContextWrappers.size()) { - case 0: - ret = NoOpDOMStoreThreePhaseCommitCohort.INSTANCE; - break; - case 1: - final Entry e = Iterables.getOnlyElement(txContextWrappers.entrySet()); - ret = createSingleCommitCohort(e.getKey(), e.getValue()); - break; - default: - ret = createMultiCommitCohort(txContextWrappers.entrySet()); + case 0: + ret = NoOpDOMStoreThreePhaseCommitCohort.INSTANCE; + break; + case 1: + final Entry e = Iterables.getOnlyElement( + txContextWrappers.entrySet()); + ret = createSingleCommitCohort(e.getKey(), e.getValue()); + break; + default: + ret = createMultiCommitCohort(txContextWrappers.entrySet()); } txContextFactory.onTransactionReady(getIdentifier(), ret.getCohortFutures()); @@ -247,6 +264,7 @@ public class TransactionProxy extends AbstractDOMStoreTransaction createSingleCommitCohort(final String shardName, final TransactionContextWrapper contextWrapper) { @@ -261,48 +279,55 @@ public class TransactionProxy extends AbstractDOMStoreTransaction getReadyOrDirectCommitFuture(TransactionContext transactionContext, - OperationCallback.Reference operationCallbackRef) { - if (transactionContext.supportsDirectCommit()) { - TransactionRateLimitingCallback rateLimitingCallback = new TransactionRateLimitingCallback( - txContextFactory.getActorContext()); - operationCallbackRef.set(rateLimitingCallback); - rateLimitingCallback.run(); - return transactionContext.directCommit(); - } else { - return transactionContext.readyTransaction(); - } + private Future getDirectCommitFuture(final TransactionContext transactionContext, + final OperationCallback.Reference operationCallbackRef, final Boolean havePermit) { + TransactionRateLimitingCallback rateLimitingCallback = new TransactionRateLimitingCallback( + txContextFactory.getActorContext()); + operationCallbackRef.set(rateLimitingCallback); + rateLimitingCallback.run(); + return transactionContext.directCommit(havePermit); } private AbstractThreePhaseCommitCohort createMultiCommitCohort( final Set> txContextWrapperEntries) { - final List> cohortFutures = new ArrayList<>(txContextWrapperEntries.size()); - for (Entry e : txContextWrapperEntries) { - LOG.debug("Tx {} Readying transaction for shard {}", getIdentifier(), e.getKey()); + final List cohorts = new ArrayList<>(txContextWrapperEntries.size()); - cohortFutures.add(e.getValue().readyTransaction()); + synchronized (GLOBAL_TX_READY_LOCK) { + for (Entry e : txContextWrapperEntries) { + LOG.debug("Tx {} Readying transaction for shard {}", getIdentifier(), e.getKey()); + + final TransactionContextWrapper wrapper = e.getValue(); + + // The remote tx version is obtained the via TransactionContext which may not be available yet so + // we pass a Supplier to dynamically obtain it. Once the ready Future is resolved the + // TransactionContext is available. + Supplier txVersionSupplier = () -> wrapper.getTransactionContext().getTransactionVersion(); + + cohorts.add(new ThreePhaseCommitCohortProxy.CohortInfo(wrapper.readyTransaction(), txVersionSupplier)); + } } - return new ThreePhaseCommitCohortProxy(txContextFactory.getActorContext(), cohortFutures, getIdentifier().toString()); + return new ThreePhaseCommitCohortProxy(txContextFactory.getActorContext(), cohorts, getIdentifier()); } - private static String shardNameFromIdentifier(final YangInstanceIdentifier path) { - return ShardStrategyFactory.getStrategy(path).findShard(path); + private String shardNameFromIdentifier(final YangInstanceIdentifier path) { + return txContextFactory.getActorContext().getShardStrategyFactory().getStrategy(path).findShard(path); } private TransactionContextWrapper getContextWrapper(final YangInstanceIdentifier path) {