X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-inmemory-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fmd%2Fsal%2Fdom%2Fstore%2Fimpl%2FInMemoryDOMDataStore.java;h=74fa73afb92f869f7cb2e945a625d489b71e71c2;hp=3e748618169889cdfcf0b9f8eb66c73e65cb4338;hb=510f3bd2d25c0e667a60ff5ef18cfbd40534cec0;hpb=4ef563c481b83e360e688a59ac346b8328870d58 diff --git a/opendaylight/md-sal/sal-inmemory-datastore/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java b/opendaylight/md-sal/sal-inmemory-datastore/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java index 3e74861816..74fa73afb9 100644 --- a/opendaylight/md-sal/sal-inmemory-datastore/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java +++ b/opendaylight/md-sal/sal-inmemory-datastore/src/main/java/org/opendaylight/controller/md/sal/dom/store/impl/InMemoryDOMDataStore.java @@ -8,14 +8,12 @@ package org.opendaylight.controller.md.sal.dom.store.impl; import static com.google.common.base.Preconditions.checkState; - import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -62,8 +60,7 @@ import org.slf4j.LoggerFactory; * to implement {@link DOMStore} contract. * */ -public class InMemoryDOMDataStore implements DOMStore, Identifiable, SchemaContextListener, - TransactionReadyPrototype,AutoCloseable { +public class InMemoryDOMDataStore extends TransactionReadyPrototype implements DOMStore, Identifiable, SchemaContextListener, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(InMemoryDOMDataStore.class); private static final ListenableFuture SUCCESSFUL_FUTURE = Futures.immediateFuture(null); @@ -82,29 +79,26 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create(); private final ListenerTree listenerTree = ListenerTree.create(); private final AtomicLong txCounter = new AtomicLong(0); - private final ListeningExecutorService listeningExecutor; private final QueuedNotificationManager, DOMImmutableDataChangeEvent> dataChangeListenerNotificationManager; private final ExecutorService dataChangeListenerExecutor; - - private final ExecutorService domStoreExecutor; + private final ListeningExecutorService commitExecutor; private final boolean debugTransactions; private final String name; private volatile AutoCloseable closeable; - public InMemoryDOMDataStore(final String name, final ExecutorService domStoreExecutor, + public InMemoryDOMDataStore(final String name, final ListeningExecutorService commitExecutor, final ExecutorService dataChangeListenerExecutor) { - this(name, domStoreExecutor, dataChangeListenerExecutor, + this(name, commitExecutor, dataChangeListenerExecutor, InMemoryDOMDataStoreConfigProperties.DEFAULT_MAX_DATA_CHANGE_LISTENER_QUEUE_SIZE, false); } - public InMemoryDOMDataStore(final String name, final ExecutorService domStoreExecutor, + public InMemoryDOMDataStore(final String name, final ListeningExecutorService commitExecutor, final ExecutorService dataChangeListenerExecutor, final int maxDataChangeListenerQueueSize, final boolean debugTransactions) { this.name = Preconditions.checkNotNull(name); - this.domStoreExecutor = Preconditions.checkNotNull(domStoreExecutor); - this.listeningExecutor = MoreExecutors.listeningDecorator(this.domStoreExecutor); + this.commitExecutor = Preconditions.checkNotNull(commitExecutor); this.dataChangeListenerExecutor = Preconditions.checkNotNull(dataChangeListenerExecutor); this.debugTransactions = debugTransactions; @@ -114,7 +108,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch "DataChangeListenerQueueMgr"); } - public void setCloseable(AutoCloseable closeable) { + public void setCloseable(final AutoCloseable closeable) { this.closeable = closeable; } @@ -123,7 +117,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch } public ExecutorService getDomStoreExecutor() { - return domStoreExecutor; + return commitExecutor; } @Override @@ -158,7 +152,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch @Override public void close() { - ExecutorServiceUtil.tryGracefulShutdown(listeningExecutor, 30, TimeUnit.SECONDS); + ExecutorServiceUtil.tryGracefulShutdown(commitExecutor, 30, TimeUnit.SECONDS); ExecutorServiceUtil.tryGracefulShutdown(dataChangeListenerExecutor, 30, TimeUnit.SECONDS); if(closeable != null) { @@ -215,80 +209,95 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch } @Override - public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction writeTx) { - LOG.debug("Tx: {} is submitted. Modifications: {}", writeTx.getIdentifier(), writeTx.getMutatedView()); - return new ThreePhaseCommitImpl(writeTx); + protected void transactionAborted(final SnapshotBackedWriteTransaction tx) { + LOG.debug("Tx: {} is closed.", tx.getIdentifier()); + } + + @Override + protected DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction tx, final DataTreeModification tree) { + LOG.debug("Tx: {} is submitted. Modifications: {}", tx.getIdentifier(), tree); + return new ThreePhaseCommitImpl(tx, tree); } private Object nextIdentifier() { return name + "-" + txCounter.getAndIncrement(); } - private class DOMStoreTransactionChainImpl implements DOMStoreTransactionChain, TransactionReadyPrototype { - + private class DOMStoreTransactionChainImpl extends TransactionReadyPrototype implements DOMStoreTransactionChain { + @GuardedBy("this") + private SnapshotBackedWriteTransaction allocatedTransaction; + @GuardedBy("this") + private DataTreeSnapshot readySnapshot; @GuardedBy("this") - private SnapshotBackedWriteTransaction latestOutstandingTx; - private boolean chainFailed = false; + @GuardedBy("this") private void checkFailed() { Preconditions.checkState(!chainFailed, "Transaction chain is failed."); } - @Override - public synchronized DOMStoreReadTransaction newReadOnlyTransaction() { - final DataTreeSnapshot snapshot; + @GuardedBy("this") + private DataTreeSnapshot getSnapshot() { checkFailed(); - if (latestOutstandingTx != null) { - checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready."); - snapshot = latestOutstandingTx.getMutatedView(); + + if (allocatedTransaction != null) { + Preconditions.checkState(readySnapshot != null, "Previous transaction %s is not ready yet", allocatedTransaction.getIdentifier()); + return readySnapshot; } else { - snapshot = dataTree.takeSnapshot(); + return dataTree.takeSnapshot(); } + } + + @GuardedBy("this") + private T recordTransaction(final T transaction) { + allocatedTransaction = transaction; + readySnapshot = null; + return transaction; + } + + @Override + public synchronized DOMStoreReadTransaction newReadOnlyTransaction() { + final DataTreeSnapshot snapshot = getSnapshot(); return new SnapshotBackedReadTransaction(nextIdentifier(), getDebugTransactions(), snapshot); } @Override public synchronized DOMStoreReadWriteTransaction newReadWriteTransaction() { - final DataTreeSnapshot snapshot; - checkFailed(); - if (latestOutstandingTx != null) { - checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready."); - snapshot = latestOutstandingTx.getMutatedView(); - } else { - snapshot = dataTree.takeSnapshot(); - } - final SnapshotBackedReadWriteTransaction ret = new SnapshotBackedReadWriteTransaction(nextIdentifier(), - getDebugTransactions(), snapshot, this); - latestOutstandingTx = ret; - return ret; + final DataTreeSnapshot snapshot = getSnapshot(); + return recordTransaction(new SnapshotBackedReadWriteTransaction(nextIdentifier(), + getDebugTransactions(), snapshot, this)); } @Override public synchronized DOMStoreWriteTransaction newWriteOnlyTransaction() { - final DataTreeSnapshot snapshot; - checkFailed(); - if (latestOutstandingTx != null) { - checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready."); - snapshot = latestOutstandingTx.getMutatedView(); - } else { - snapshot = dataTree.takeSnapshot(); + final DataTreeSnapshot snapshot = getSnapshot(); + return recordTransaction(new SnapshotBackedWriteTransaction(nextIdentifier(), + getDebugTransactions(), snapshot, this)); + } + + @Override + protected synchronized void transactionAborted(final SnapshotBackedWriteTransaction tx) { + if (tx.equals(allocatedTransaction)) { + Preconditions.checkState(readySnapshot == null, "Unexpected abort of transaction %s with ready snapshot %s", tx, readySnapshot); + allocatedTransaction = null; } - final SnapshotBackedWriteTransaction ret = new SnapshotBackedWriteTransaction(nextIdentifier(), - getDebugTransactions(), snapshot, this); - latestOutstandingTx = ret; - return ret; } @Override - public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction tx) { - DOMStoreThreePhaseCommitCohort storeCohort = InMemoryDOMDataStore.this.ready(tx); - return new ChainedTransactionCommitImpl(tx, storeCohort, this); + protected synchronized DOMStoreThreePhaseCommitCohort transactionReady(final SnapshotBackedWriteTransaction tx, final DataTreeModification tree) { + Preconditions.checkState(tx.equals(allocatedTransaction), "Mis-ordered ready transaction %s last allocated was %s", tx, allocatedTransaction); + if (readySnapshot != null) { + // The snapshot should have been cleared + LOG.warn("Uncleared snapshot {} encountered, overwritten with transaction {} snapshot {}", readySnapshot, tx, tree); + } + + final DOMStoreThreePhaseCommitCohort cohort = InMemoryDOMDataStore.this.transactionReady(tx, tree); + readySnapshot = tree; + return new ChainedTransactionCommitImpl(tx, cohort, this); } @Override public void close() { - // FIXME: this call doesn't look right here - listeningExecutor is shared and owned // by the outer class. //listeningExecutor.shutdownNow(); @@ -297,31 +306,30 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch protected synchronized void onTransactionFailed(final SnapshotBackedWriteTransaction transaction, final Throwable t) { chainFailed = true; - } public synchronized void onTransactionCommited(final SnapshotBackedWriteTransaction transaction) { - // If committed transaction is latestOutstandingTx we clear - // latestOutstandingTx - // field in order to base new transactions on Datastore Data Tree - // directly. - if (transaction.equals(latestOutstandingTx)) { - latestOutstandingTx = null; + // If the committed transaction was the one we allocated last, + // we clear it and the ready snapshot, so the next transaction + // allocated refers to the data tree directly. + if (transaction.equals(allocatedTransaction)) { + if (readySnapshot == null) { + LOG.warn("Transaction {} committed while no ready snapshot present", transaction); + } + + allocatedTransaction = null; + readySnapshot = null; } } - } private static class ChainedTransactionCommitImpl implements DOMStoreThreePhaseCommitCohort { - private final SnapshotBackedWriteTransaction transaction; private final DOMStoreThreePhaseCommitCohort delegate; - private final DOMStoreTransactionChainImpl txChain; protected ChainedTransactionCommitImpl(final SnapshotBackedWriteTransaction transaction, final DOMStoreThreePhaseCommitCohort delegate, final DOMStoreTransactionChainImpl txChain) { - super(); this.transaction = transaction; this.delegate = delegate; this.txChain = txChain; @@ -355,29 +363,26 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch public void onSuccess(final Void result) { txChain.onTransactionCommited(transaction); } - }); return commitFuture; } - } private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort { - private final SnapshotBackedWriteTransaction transaction; private final DataTreeModification modification; private ResolveDataChangeEventsTask listenerResolver; private DataTreeCandidate candidate; - public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction) { + public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction, final DataTreeModification modification) { this.transaction = writeTransaction; - this.modification = transaction.getMutatedView(); + this.modification = modification; } @Override public ListenableFuture canCommit() { - return listeningExecutor.submit(new Callable() { + return commitExecutor.submit(new Callable() { @Override public Boolean call() throws TransactionCommitFailedException { try { @@ -401,7 +406,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch @Override public ListenableFuture preCommit() { - return listeningExecutor.submit(new Callable() { + return commitExecutor.submit(new Callable() { @Override public Void call() { candidate = dataTree.prepare(modification); @@ -425,7 +430,7 @@ public class InMemoryDOMDataStore implements DOMStore, Identifiable, Sch * The commit has to occur atomically with regard to listener * registrations. */ - synchronized (this) { + synchronized (InMemoryDOMDataStore.this) { dataTree.commit(candidate); listenerResolver.resolve(dataChangeListenerNotificationManager); }