BUG-5280: switch transactionIdentifier 62/38962/34
authorRobert Varga <rovarga@cisco.com>
Tue, 17 May 2016 08:29:38 +0000 (10:29 +0200)
committerRobert Varga <rovarga@cisco.com>
Wed, 8 Jun 2016 09:00:49 +0000 (11:00 +0200)
This eliminates datastore-internal TransactionIdentifier in favor of the
concept one, which is structured. Since the structured identifier also
includes LocalHistoryIdentifier, this eliminates also ChainedTransactionIdentifier.

Change-Id: Iabfa2ddd7aadd1e4913115f342e520ffaa8b84f0
Signed-off-by: Robert Varga <rovarga@cisco.com>
35 files changed:
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionContext.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionContextFactory.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DebugThreePhaseCommitCohort.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DistributedDataStore.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/LocalThreePhaseCommitCohort.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/LocalTransactionChain.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/LocalTransactionContext.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/LocalTransactionFactory.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/LocalTransactionFactoryImpl.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpTransactionContext.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/OperationLimiter.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/RemoteTransactionContext.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/RemoteTransactionContextSupport.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/SingleCommitCohortProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionChainProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContextFactory.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContextWrapper.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionReadyReplyMapper.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShardCommitCoordinator.java
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifier.java [deleted file]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifier.java [deleted file]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionIdentifier.java [deleted file]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/TransactionIdentifierUtils.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/DebugThreePhaseCommitCohortTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/LocalTransactionContextTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/MockIdentifiers.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/OperationLimiterTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/TransactionChainProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/TransactionContextWrapperTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/TransactionProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/compat/PreBoronTransactionProxyTest.java
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifierTest.java [deleted file]
opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifierTest.java [deleted file]

index 7b25abbe836fd0cd655ad3c24d6d7b3f881bac45..057dbfa56abd0f08f10477eba6c93df42afd46de 100644 (file)
@@ -8,7 +8,7 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import javax.annotation.Nonnull;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -23,8 +23,7 @@ abstract class AbstractTransactionContext implements TransactionContext {
         this(transactionIdentifier, DataStoreVersions.CURRENT_VERSION);
     }
 
-    protected AbstractTransactionContext(TransactionIdentifier transactionIdentifier,
-            short transactionVersion) {
+    protected AbstractTransactionContext(TransactionIdentifier transactionIdentifier, short transactionVersion) {
         this.transactionIdentifier = transactionIdentifier;
         this.transactionVersion = transactionVersion;
     }
index 4832d8a6af5f17bbfa9d3321d37e39dea3ad6229..b5afd596bf3b1ded0abdbaceb9c80e309ccaacda 100644 (file)
@@ -14,10 +14,11 @@ import java.util.Collection;
 import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
 import javax.annotation.Nonnull;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
@@ -35,28 +36,37 @@ import scala.util.Try;
  */
 abstract class AbstractTransactionContextFactory<F extends LocalTransactionFactory> implements AutoCloseable {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractTransactionContextFactory.class);
-    private static final MemberName UNKNOWN_MEMBER = MemberName.forName("UNKNOWN-MEMBER");
-
-    protected static final AtomicLong TX_COUNTER = new AtomicLong();
+    @SuppressWarnings("rawtypes")
+    private static final AtomicLongFieldUpdater<AbstractTransactionContextFactory> TX_COUNTER_UPDATER =
+            AtomicLongFieldUpdater.newUpdater(AbstractTransactionContextFactory.class, "nextTx");
 
     private final ConcurrentMap<String, F> knownLocal = new ConcurrentHashMap<>();
+    private final LocalHistoryIdentifier historyId;
     private final ActorContext actorContext;
 
-    protected AbstractTransactionContextFactory(final ActorContext actorContext) {
+    // Used via TX_COUNTER_UPDATER
+    @SuppressWarnings("unused")
+    private volatile long nextTx;
+
+    protected AbstractTransactionContextFactory(final ActorContext actorContext,
+            final LocalHistoryIdentifier historyId) {
         this.actorContext = Preconditions.checkNotNull(actorContext);
+        this.historyId = Preconditions.checkNotNull(historyId);
     }
 
     final ActorContext getActorContext() {
         return actorContext;
     }
 
+    final LocalHistoryIdentifier getHistoryId() {
+        return historyId;
+    }
+
     private TransactionContext maybeCreateLocalTransactionContext(final TransactionProxy parent, final String shardName) {
         final LocalTransactionFactory local = knownLocal.get(shardName);
         if (local != null) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} - Creating local component for shard {} using factory {}",
-                        parent.getIdentifier(), shardName, local);
-            }
+            LOG.debug("Tx {} - Creating local component for shard {} using factory {}", parent.getIdentifier(),
+                shardName, local);
 
             try {
                 return createLocalTransactionContext(local, parent);
@@ -147,9 +157,8 @@ abstract class AbstractTransactionContextFactory<F extends LocalTransactionFacto
         }
     }
 
-    protected MemberName getMemberName() {
-        final MemberName ret = getActorContext().getCurrentMemberName();
-        return ret == null ? UNKNOWN_MEMBER : ret;
+    protected final MemberName getMemberName() {
+        return historyId.getClientId().getFrontendId().getMemberName();
     }
 
     /**
@@ -157,7 +166,9 @@ abstract class AbstractTransactionContextFactory<F extends LocalTransactionFacto
      * factory.
      * @return Transaction identifier, may not be null.
      */
-    protected abstract TransactionIdentifier nextIdentifier();
+    protected final TransactionIdentifier nextIdentifier() {
+        return new TransactionIdentifier(historyId, TX_COUNTER_UPDATER.getAndIncrement(this));
+    }
 
     /**
      * Find the primary shard actor.
index 9a15de8d06168181814921a01986bcbfce1d014e..e0b5c1f9c869ce72e283708f45e620341512ba8b 100644 (file)
@@ -13,7 +13,7 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import java.util.List;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
index fe321d4cd2808bff7c6be29b67222b4876c8b2fe..0bdb29b598a56e876b2acddfa2ee38186c482125 100644 (file)
@@ -118,7 +118,7 @@ public class DistributedDataStore implements DistributedDataStoreInterface, Sche
         this.waitTillReadyTimeInMillis =
                 actorContext.getDatastoreContext().getShardLeaderElectionTimeout().duration().toMillis() * READY_WAIT_FACTOR;
 
-        this.txContextFactory = TransactionContextFactory.create(actorContext);
+        this.txContextFactory = new TransactionContextFactory(actorContext, identifier);
 
         datastoreConfigMXBean = new DatastoreConfigurationMXBeanImpl(
                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
@@ -135,7 +135,7 @@ public class DistributedDataStore implements DistributedDataStoreInterface, Sche
         this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
         this.client = null;
         this.identifier = Preconditions.checkNotNull(identifier);
-        this.txContextFactory = TransactionContextFactory.create(actorContext);
+        this.txContextFactory = new TransactionContextFactory(actorContext, identifier);
         this.waitTillReadyTimeInMillis =
                 actorContext.getDatastoreContext().getShardLeaderElectionTimeout().duration().toMillis() * READY_WAIT_FACTOR;
     }
index ec7b46d5bbd5ef278489560d534728999820ef65..d90b82be4bff1184bd0e846c6fe82f13e5c814aa 100644 (file)
@@ -12,10 +12,11 @@ import akka.dispatch.Futures;
 import akka.dispatch.OnComplete;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListenableFuture;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.TransactionIdentifierUtils;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.SnapshotBackedWriteTransaction;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
@@ -62,8 +63,8 @@ class LocalThreePhaseCommitCohort implements DOMStoreThreePhaseCommitCohort {
             return Futures.failed(operationError);
         }
 
-        final ReadyLocalTransaction message = new ReadyLocalTransaction(transaction.getIdentifier().toString(),
-                modification, immediate);
+        final ReadyLocalTransaction message = new ReadyLocalTransaction(
+            TransactionIdentifierUtils.actorNameFor(transaction.getIdentifier()), modification, immediate);
         return actorContext.executeOperationAsync(leader, message, actorContext.getTransactionCommitOperationTimeout());
     }
 
index b42c52d99341a716ae337510cea158cac1fd6de4..2e329dd15e07856cdd39b45202d2871ca1e72d87 100644 (file)
@@ -11,7 +11,7 @@ import akka.actor.ActorSelection;
 import com.google.common.base.Preconditions;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.sal.core.spi.data.AbstractSnapshotBackedTransactionChain;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
@@ -59,7 +59,9 @@ final class LocalTransactionChain extends AbstractSnapshotBackedTransactionChain
     }
 
     @Override
-    protected DOMStoreThreePhaseCommitCohort createCohort(final SnapshotBackedWriteTransaction<TransactionIdentifier> transaction, final DataTreeModification modification) {
+    protected DOMStoreThreePhaseCommitCohort createCohort(
+            final SnapshotBackedWriteTransaction<TransactionIdentifier> transaction,
+            final DataTreeModification modification) {
         return new LocalChainThreePhaseCommitCohort(transaction, modification);
     }
 
index 55ff68e39f9d6e3626000efde345d7b4386db0fc..0dbd2a284def03ae2a722dc9b7fbef62e69dbe23 100644 (file)
@@ -12,7 +12,7 @@ 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.SettableFuture;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.AbstractRead;
 import org.opendaylight.controller.cluster.datastore.modification.AbstractModification;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
index ff0ef76cec34084097442010c6702c93b9c7b582..a08fef8eb1bc79434fec03510d5e40771bb21671 100644 (file)
@@ -7,7 +7,7 @@
  */
 package org.opendaylight.controller.cluster.datastore;
 
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
index 700d96f4fd7d79e2e9b513215c93e317cfe27dc0..7c05fef87fc6013aebeae533697fa6309bd9f8a8 100644 (file)
@@ -11,7 +11,7 @@ import akka.actor.ActorSelection;
 import com.google.common.base.Preconditions;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
index b3dd638e56708b6f77126c8e1bbbe288a57b2cac..3f5a1521dc4332801d40302849fe092f377f3fd7 100644 (file)
@@ -9,8 +9,8 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorSelection;
 import com.google.common.util.concurrent.SettableFuture;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.AbstractRead;
 import org.opendaylight.controller.cluster.datastore.modification.AbstractModification;
 import org.opendaylight.controller.md.sal.common.api.data.DataStoreUnavailableException;
index 22413badc8e91f3f5dae52dc5c44622e7de76fcd..ad26d0a50b4d19dccc3afee4086ea6fb16c86726 100644 (file)
@@ -12,7 +12,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
index 29783a6e7dd4c19cb75971e4732cb2d57e018042..75c5de82c328a3a2e1255a17184fe5f0d79d88ee 100644 (file)
@@ -13,7 +13,7 @@ import akka.dispatch.OnComplete;
 import akka.util.Timeout;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.SettableFuture;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.AbstractRead;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
@@ -21,6 +21,7 @@ import org.opendaylight.controller.cluster.datastore.messages.SerializableMessag
 import org.opendaylight.controller.cluster.datastore.modification.AbstractModification;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.TransactionIdentifierUtils;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -105,8 +106,8 @@ public class RemoteTransactionContext extends AbstractTransactionContext {
     }
 
     private BatchedModifications newBatchedModifications() {
-        return new BatchedModifications(getIdentifier().toString(), getTransactionVersion(),
-                getIdentifier().getChainId());
+        return new BatchedModifications(TransactionIdentifierUtils.actorNameFor(getIdentifier()),
+            getTransactionVersion(), RemoteTransactionContextSupport.compatTransactionChainId(getIdentifier()));
     }
 
     private void batchModification(Modification modification) {
index 4a031fa9160d55ae341e2ea50a4539513dc7be96..4f41d8902e029dc97b99b9cb385e47682e076f7b 100644 (file)
@@ -14,13 +14,14 @@ import akka.pattern.AskTimeoutException;
 import akka.util.Timeout;
 import com.google.common.base.Preconditions;
 import java.util.concurrent.TimeUnit;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.TransactionIdentifierUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
@@ -116,6 +117,15 @@ final class RemoteTransactionContextSupport {
         }
     }
 
+    /**
+     * @deprecated Temporary utility for extracting transaction chain ID from a {@link TransactionIdentifier}
+     */
+    @Deprecated
+    static String compatTransactionChainId(final TransactionIdentifier txId) {
+        final long historyId = txId.getHistoryId().getHistoryId();
+        return historyId == 0 ? "" : Long.toUnsignedString(historyId);
+    }
+
     /**
      * Performs a CreateTransaction try async.
      */
@@ -125,8 +135,8 @@ final class RemoteTransactionContextSupport {
                     primaryShardInfo.getPrimaryShardActor());
         }
 
-        Object serializedCreateMessage = new CreateTransaction(getIdentifier().toString(),
-                getTransactionType().ordinal(), getIdentifier().getChainId(),
+        Object serializedCreateMessage = new CreateTransaction(TransactionIdentifierUtils.actorNameFor(getIdentifier()),
+                getTransactionType().ordinal(), compatTransactionChainId(getIdentifier()),
                     primaryShardInfo.getPrimaryShardVersion()).toSerializable();
 
         Future<Object> createTxFuture = getActorContext().executeOperationAsync(
index 9c17bc1a476c3c95060498656df9061a8312bcab..505c959337840dfc9d75de1178d3d08c27af02a2 100644 (file)
@@ -8,10 +8,12 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.dispatch.OnComplete;
+import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
 import java.util.Arrays;
 import java.util.List;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.slf4j.Logger;
@@ -30,15 +32,15 @@ class SingleCommitCohortProxy extends AbstractThreePhaseCommitCohort<Object> {
 
     private final ActorContext actorContext;
     private final Future<Object> cohortFuture;
-    private final String transactionId;
+    private final TransactionIdentifier transactionId;
     private volatile DOMStoreThreePhaseCommitCohort delegateCohort = NoOpDOMStoreThreePhaseCommitCohort.INSTANCE;
     private final OperationCallback.Reference operationCallbackRef;
 
-    SingleCommitCohortProxy(ActorContext actorContext, Future<Object> cohortFuture, String transactionId,
+    SingleCommitCohortProxy(ActorContext actorContext, Future<Object> cohortFuture, TransactionIdentifier transactionId,
             OperationCallback.Reference operationCallbackRef) {
         this.actorContext = actorContext;
         this.cohortFuture = cohortFuture;
-        this.transactionId = transactionId;
+        this.transactionId = Preconditions.checkNotNull(transactionId);
         this.operationCallbackRef = operationCallbackRef;
     }
 
index cf00d5fa39e3d317fe99af05d2f30f112fcb6f22..b93f540f20783b74288b7112056883d6a535c9cf 100644 (file)
@@ -17,12 +17,11 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.function.Function;
 import javax.annotation.Nonnull;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionChainIdentifier;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainClosedException;
@@ -120,11 +119,9 @@ final class TransactionChainProxy extends AbstractTransactionContextFactory<Loca
     };
 
     private static final Logger LOG = LoggerFactory.getLogger(TransactionChainProxy.class);
-    private static final AtomicInteger CHAIN_COUNTER = new AtomicInteger();
     private static final AtomicReferenceFieldUpdater<TransactionChainProxy, State> STATE_UPDATER =
             AtomicReferenceFieldUpdater.newUpdater(TransactionChainProxy.class, State.class, "currentState");
 
-    private final TransactionChainIdentifier transactionChainId;
     private final TransactionContextFactory parent;
     private volatile State currentState = IDLE_STATE;
 
@@ -149,15 +146,13 @@ final class TransactionChainProxy extends AbstractTransactionContextFactory<Loca
      */
     private final ConcurrentMap<TransactionIdentifier, Promise<Object>> priorReadOnlyTxPromises = new ConcurrentHashMap<>();
 
-    TransactionChainProxy(final TransactionContextFactory parent) {
-        super(parent.getActorContext());
-
-        transactionChainId = new TransactionChainIdentifier(parent.getActorContext().getCurrentMemberName(), CHAIN_COUNTER.incrementAndGet());
+    TransactionChainProxy(final TransactionContextFactory parent, final LocalHistoryIdentifier historyId) {
+        super(parent.getActorContext(), historyId);
         this.parent = parent;
     }
 
     public String getTransactionChainId() {
-        return transactionChainId.toString();
+        return getHistoryId().toString();
     }
 
     @Override
@@ -189,7 +184,7 @@ final class TransactionChainProxy extends AbstractTransactionContextFactory<Loca
         getActorContext().broadcast(new Function<Short, Object>() {
             @Override
             public Object apply(Short version) {
-                return new CloseTransactionChain(transactionChainId.toString(), version).toSerializable();
+                return new CloseTransactionChain(getHistoryId().toString(), version).toSerializable();
             }
         });
     }
@@ -337,9 +332,4 @@ final class TransactionChainProxy extends AbstractTransactionContextFactory<Loca
             promise.success(null);
         }
     }
-
-    @Override
-    protected TransactionIdentifier nextIdentifier() {
-        return transactionChainId.newTransactionIdentifier();
-    }
 }
index db8dedcf353c3450e5475dc5cb2c8723dd401f0a..8c4f6b3456c63a8db9697cf1c9a277dbae9eb01b 100644 (file)
@@ -9,7 +9,10 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorSelection;
 import java.util.Collection;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import java.util.concurrent.atomic.AtomicLong;
+import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
@@ -21,24 +24,16 @@ import scala.concurrent.Future;
  * transactions (ie not chained).
  */
 final class TransactionContextFactory extends AbstractTransactionContextFactory<LocalTransactionFactoryImpl> {
+    private final AtomicLong nextHistory = new AtomicLong(1);
 
-    private TransactionContextFactory(final ActorContext actorContext) {
-        super(actorContext);
-    }
-
-    static TransactionContextFactory create(final ActorContext actorContext) {
-        return new TransactionContextFactory(actorContext);
+    TransactionContextFactory(final ActorContext actorContext, final ClientIdentifier clientId) {
+        super(actorContext, new LocalHistoryIdentifier(clientId, 0));
     }
 
     @Override
     public void close() {
     }
 
-    @Override
-    protected TransactionIdentifier nextIdentifier() {
-        return TransactionIdentifier.create(getMemberName(), TX_COUNTER.getAndIncrement());
-    }
-
     @Override
     protected LocalTransactionFactoryImpl factoryForShard(final String shardName, final ActorSelection shardLeader, final DataTree dataTree) {
         return new LocalTransactionFactoryImpl(getActorContext(), shardLeader, dataTree);
@@ -55,10 +50,11 @@ final class TransactionContextFactory extends AbstractTransactionContextFactory<
     }
 
     DOMStoreTransactionChain createTransactionChain() {
-        return new TransactionChainProxy(this);
+        return new TransactionChainProxy(this, new LocalHistoryIdentifier(getHistoryId().getClientId(),
+                nextHistory.getAndIncrement()));
     }
 
     @Override
-    protected void onTransactionContextCreated(TransactionIdentifier transactionId) {
+    protected void onTransactionContextCreated(final TransactionIdentifier transactionId) {
     }
 }
index e8dab2c17ebb12e54241543f7ace660e2940009f..ef9ee68bf016df46e5dfcd1711d42adcc20f4664 100644 (file)
@@ -16,7 +16,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.concurrent.GuardedBy;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,7 +48,7 @@ class TransactionContextWrapper {
 
     private final OperationLimiter limiter;
 
-    TransactionContextWrapper(TransactionIdentifier identifier, final ActorContext actorContext) {
+    TransactionContextWrapper(final TransactionIdentifier identifier, final ActorContext actorContext) {
         this.identifier = Preconditions.checkNotNull(identifier);
         this.limiter = new OperationLimiter(identifier,
                 actorContext.getDatastoreContext().getShardBatchedModificationCount() + 1, // 1 extra permit for the ready operation
index b03398093c9ddcd838b8121b42f94a026acf4e4c..ddc9912c95a61d1eaf522a8b430bd874e063415b 100644 (file)
@@ -25,7 +25,7 @@ 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.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;
@@ -35,6 +35,7 @@ import org.opendaylight.controller.cluster.datastore.modification.MergeModificat
 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.cluster.datastore.utils.TransactionIdentifierUtils;
 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;
@@ -263,8 +264,8 @@ public class TransactionProxy extends AbstractDOMStoreTransaction<TransactionIde
             future = getDirectCommitFuture(transactionContext, operationCallbackRef);
         }
 
-        return new SingleCommitCohortProxy(txContextFactory.getActorContext(), future, getIdentifier().toString(),
-                operationCallbackRef);
+        return new SingleCommitCohortProxy(txContextFactory.getActorContext(), future, getIdentifier(),
+            operationCallbackRef);
     }
 
     private Future<?> getDirectCommitFuture(TransactionContext transactionContext,
@@ -299,7 +300,7 @@ public class TransactionProxy extends AbstractDOMStoreTransaction<TransactionIde
         }
 
         return new ThreePhaseCommitCohortProxy(txContextFactory.getActorContext(), cohorts,
-                getIdentifier().toString());
+                TransactionIdentifierUtils.actorNameFor(getIdentifier()));
     }
 
     private String shardNameFromIdentifier(final YangInstanceIdentifier path) {
index 22bfbb1316ae674e9807936644d1b107a55a9cdc..43c3be11c278de94cddfdccb66bcc26414bf1f5f 100644 (file)
@@ -10,7 +10,7 @@ package org.opendaylight.controller.cluster.datastore;
 import akka.actor.ActorSelection;
 import akka.dispatch.Mapper;
 import com.google.common.base.Preconditions;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.slf4j.Logger;
index 9cbcbf9dea73c0750ea51603eee701db6ec42cfb..e906ce116043c8368894f65810d147df2f17c3b7 100644 (file)
@@ -10,16 +10,22 @@ package org.opendaylight.controller.cluster.datastore.entityownership;
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
 import akka.actor.Status.Failure;
+import com.google.common.base.Preconditions;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Queue;
+import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.FrontendIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.FrontendType;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.DataStoreVersions;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
+import org.opendaylight.controller.cluster.datastore.utils.TransactionIdentifierUtils;
 import org.slf4j.Logger;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -35,17 +41,20 @@ class EntityOwnershipShardCommitCoordinator {
             return "entityCommitRetry";
         }
     };
+    private static final FrontendType FRONTEND_TYPE = FrontendType.forName("entity-ownership-internal");
 
-    private final Logger log;
-    private int transactionIDCounter = 0;
-    private final MemberName localMemberName;
     private final Queue<Modification> pendingModifications = new LinkedList<>();
+    private final LocalHistoryIdentifier historyId;
+    private final Logger log;
+
     private BatchedModifications inflightCommit;
     private Cancellable retryCommitSchedule;
+    private long transactionIDCounter = 0;
 
     EntityOwnershipShardCommitCoordinator(MemberName localMemberName, Logger log) {
-        this.localMemberName = localMemberName;
-        this.log = log;
+        this.log = Preconditions.checkNotNull(log);
+        historyId = new LocalHistoryIdentifier(
+                ClientIdentifier.create(FrontendIdentifier.create(localMemberName, FRONTEND_TYPE), 0), 0);
     }
 
     boolean handleMessage(Object message, EntityOwnershipShard shard) {
@@ -194,9 +203,8 @@ class EntityOwnershipShardCommitCoordinator {
     }
 
     BatchedModifications newBatchedModifications() {
-        BatchedModifications modifications = new BatchedModifications(
-                TransactionIdentifier.create(localMemberName, ++transactionIDCounter).toString(),
-                DataStoreVersions.CURRENT_VERSION, "");
+        BatchedModifications modifications = new BatchedModifications(TransactionIdentifierUtils.actorNameFor(
+            new TransactionIdentifier(historyId, ++transactionIDCounter)), DataStoreVersions.CURRENT_VERSION, "");
         modifications.setDoCommitOnReady(true);
         modifications.setReady(true);
         modifications.setTotalMessagesSent(1);
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifier.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifier.java
deleted file mode 100644 (file)
index 4753437..0000000
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-package org.opendaylight.controller.cluster.datastore.identifiers;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
-
-/**
- * A TransactionIdentifier which is tied to a backend transaction chain.
- */
-public class ChainedTransactionIdentifier extends TransactionIdentifier {
-    private final String chainId;
-    private Supplier<String> stringRepresentation;
-
-    public ChainedTransactionIdentifier(final TransactionChainIdentifier chainId, final long txnCounter) {
-        super(chainId.getMemberName(), txnCounter);
-        Preconditions.checkNotNull(chainId);
-        this.chainId = chainId.toString();
-        stringRepresentation = Suppliers.memoize(new Supplier<String>() {
-            @Override
-            public String get() {
-                return new StringBuilder(chainId.toString().length() + TX_SEPARATOR.length() + 21).
-                        append(chainId).append(TX_SEPARATOR).append(getCounter()).append('-').
-                        append(getTimestamp()).toString();
-            }
-        });
-    }
-
-
-    @Override
-    public String getChainId() {
-        return chainId;
-    }
-
-    @Override
-    public String toString() {
-        return stringRepresentation.get();
-    }
-
-}
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifier.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifier.java
deleted file mode 100644 (file)
index 851e23c..0000000
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.datastore.identifiers;
-
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
-import java.util.concurrent.atomic.AtomicLong;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-
-public class TransactionChainIdentifier {
-
-    protected static final String CHAIN_SEPARATOR = "-chn-";
-
-    private final AtomicLong txnCounter = new AtomicLong();
-    private final Supplier<String> stringRepresentation;
-    private final MemberName memberName;
-
-    public TransactionChainIdentifier(final MemberName memberName, final long counter) {
-        this.memberName = memberName;
-        stringRepresentation = Suppliers.memoize(() -> {
-            final StringBuilder sb = new StringBuilder();
-            sb.append(memberName.getName()).append(CHAIN_SEPARATOR);
-            sb.append(counter);
-            return sb.toString();
-        });
-    }
-    @Override
-    public String toString() {
-        return stringRepresentation.get();
-    }
-
-    public TransactionIdentifier newTransactionIdentifier(){
-        return new ChainedTransactionIdentifier(this, txnCounter.incrementAndGet());
-    }
-
-    public MemberName getMemberName() {
-        return memberName;
-    }
-}
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionIdentifier.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionIdentifier.java
deleted file mode 100644 (file)
index 1e86203..0000000
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.datastore.identifiers;
-
-import com.google.common.base.Preconditions;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-
-public class TransactionIdentifier {
-    protected static final String TX_SEPARATOR = "-txn-";
-
-    private final MemberName memberName;
-    private final long counter;
-    private final long timestamp;
-    private String stringRepresentation;
-
-    public TransactionIdentifier(MemberName memberName, long counter) {
-        this.memberName = Preconditions.checkNotNull(memberName, "memberName should not be null");
-        this.counter = counter;
-        this.timestamp = System.currentTimeMillis();
-    }
-
-    public String getChainId() {
-        return "";
-    }
-
-    protected MemberName getMemberName() {
-        return memberName;
-    }
-
-    protected long getCounter() {
-        return counter;
-    }
-
-    protected long getTimestamp() {
-        return timestamp;
-    }
-
-    public static TransactionIdentifier create(MemberName memberName, long counter) {
-        return new TransactionIdentifier(memberName, counter);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-
-        TransactionIdentifier that = (TransactionIdentifier) o;
-
-        if (counter != that.counter) {
-            return false;
-        }
-
-        if (timestamp != that.timestamp) {
-            return false;
-        }
-
-        if (!memberName.equals(that.memberName)) {
-            return false;
-        }
-
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int result = memberName.hashCode();
-        result = 31 * result + (int) (counter ^ (counter >>> 32));
-        result = 31 * result + (int)(timestamp ^ (timestamp >>> 32));
-        return result;
-    }
-
-
-    @Override
-    public String toString() {
-        if(stringRepresentation == null) {
-            stringRepresentation = new StringBuilder(memberName.getName().length() + TX_SEPARATOR.length() + 21).
-                append(memberName.getName()).append(TX_SEPARATOR).append(counter).append('-').append(timestamp).toString();
-        }
-
-        return stringRepresentation;
-    }
-
-}
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/TransactionIdentifierUtils.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/TransactionIdentifierUtils.java
new file mode 100644 (file)
index 0000000..11daa47
--- /dev/null
@@ -0,0 +1,35 @@
+/*
+ * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore.utils;
+
+import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.FrontendIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
+
+public final class TransactionIdentifierUtils {
+    private TransactionIdentifierUtils() {
+        throw new UnsupportedOperationException();
+    }
+
+    public static String actorNameFor(final TransactionIdentifier txId) {
+        final LocalHistoryIdentifier historyId = txId.getHistoryId();
+        final ClientIdentifier clientId = historyId.getClientId();
+        final FrontendIdentifier frontendId = clientId.getFrontendId();
+
+        final StringBuilder sb = new StringBuilder();
+        sb.append(frontendId.getMemberName().getName()).append(':');
+        sb.append(frontendId.getClientType().getName()).append('@');
+        sb.append(clientId.getGeneration()).append(':');
+        if (historyId.getHistoryId() != 0) {
+            sb.append(historyId.getHistoryId()).append('-');
+        }
+
+        return sb.append(txId.getTransactionId()).toString();
+    }
+}
index f7a52035f5ee9ac00e4cee15c4239e9551aefc5d..0fa716a7ffb5eab6a52c5ffd3a9f3f5e40680772 100644 (file)
@@ -45,6 +45,7 @@ import org.mockito.ArgumentMatcher;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
+import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
 import org.opendaylight.controller.cluster.datastore.TransactionProxyTest.TestException;
@@ -171,7 +172,8 @@ public abstract class AbstractTransactionProxyTest {
         doReturn(mockClusterWrapper).when(mockActorContext).getClusterWrapper();
         doReturn(dataStoreContextBuilder.build()).when(mockActorContext).getDatastoreContext();
 
-        mockComponentFactory = TransactionContextFactory.create(mockActorContext);
+        final ClientIdentifier mockClientId = MockIdentifiers.clientIdentifier(getClass(), memberName);
+        mockComponentFactory = new TransactionContextFactory(mockActorContext, mockClientId);
 
         Timer timer = new MetricRegistry().timer("test");
         doReturn(timer).when(mockActorContext).getOperationTimer(any(String.class));
@@ -188,7 +190,7 @@ public abstract class AbstractTransactionProxyTest {
             public boolean matches(Object argument) {
                 if(CreateTransaction.class.equals(argument.getClass())) {
                     CreateTransaction obj = CreateTransaction.fromSerializable(argument);
-                    return obj.getTransactionId().startsWith(memberName) &&
+                    return obj.getTransactionId().startsWith(memberName + ':') &&
                             obj.getTransactionType() == type.ordinal();
                 }
 
index bee1704fabe446941dae00ea8b3a27a39c1e8e85..43fbfea6121d0d1b70d809c5d2d70f891984563a 100644 (file)
@@ -21,8 +21,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import java.util.ArrayList;
 import java.util.List;
 import org.junit.Test;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.slf4j.Logger;
 import scala.concurrent.Future;
 
@@ -32,6 +31,8 @@ import scala.concurrent.Future;
  * @author Thomas Pantelis
  */
 public class DebugThreePhaseCommitCohortTest {
+    private final TransactionIdentifier transactionId = MockIdentifiers.transactionIdentifier(
+        DebugThreePhaseCommitCohortTest.class, "mock");
 
     @Test
     public void test() {
@@ -48,9 +49,8 @@ public class DebugThreePhaseCommitCohortTest {
         List<Future<Object>> expCohortFutures = new ArrayList<>();
         doReturn(expCohortFutures).when(mockDelegate).getCohortFutures();
 
-        TransactionIdentifier transactionId = TransactionIdentifier.create(MemberName.forName("1"), 1);
         Throwable debugContext = new RuntimeException("mock");
-        DebugThreePhaseCommitCohort cohort = new DebugThreePhaseCommitCohort(transactionId , mockDelegate , debugContext );
+        DebugThreePhaseCommitCohort cohort = new DebugThreePhaseCommitCohort(transactionId , mockDelegate , debugContext);
 
         Logger mockLogger = mock(Logger.class);
         cohort.setLogger(mockLogger);
index dd81b6bc68a1c80d71f676eb7a5e6bc871b9f2d8..e32d0e4642ebd3efc8cbf09199174010a7c8a4e2 100644 (file)
@@ -21,7 +21,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.DataExists;
 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
@@ -37,18 +36,15 @@ import scala.concurrent.Future;
 public class LocalTransactionContextTest {
 
     @Mock
-    OperationLimiter limiter;
+    private OperationLimiter limiter;
 
     @Mock
-    TransactionIdentifier identifier;
+    private DOMStoreReadWriteTransaction readWriteTransaction;
 
     @Mock
-    DOMStoreReadWriteTransaction readWriteTransaction;
+    private LocalTransactionReadySupport mockReadySupport;
 
-    @Mock
-    LocalTransactionReadySupport mockReadySupport;
-
-    LocalTransactionContext localTransactionContext;
+    private LocalTransactionContext localTransactionContext;
 
     @Before
     public void setUp() {
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/MockIdentifiers.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/MockIdentifiers.java
new file mode 100644 (file)
index 0000000..a4062c1
--- /dev/null
@@ -0,0 +1,30 @@
+/*
+ * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.FrontendIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.FrontendType;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
+
+public final class MockIdentifiers {
+    public static ClientIdentifier clientIdentifier(final Class<?> clazz, final String memberName) {
+        return ClientIdentifier.create(FrontendIdentifier.create(MemberName.forName(memberName),
+            FrontendType.forName(clazz.getSimpleName())), 0);
+    }
+
+    public static LocalHistoryIdentifier historyIdentifier(final Class<?> clazz, final String memberName) {
+        return new LocalHistoryIdentifier(clientIdentifier(clazz, memberName), 0);
+    }
+
+    public static TransactionIdentifier transactionIdentifier(final Class<?> clazz, final String memberName) {
+        return new TransactionIdentifier(historyIdentifier(clazz, memberName), 0);
+    };
+}
index 31a8776e5b86e09aba774c07951acbf2163add33..91eda66561fddc1b165c3454a0866bb3419820ed 100644 (file)
@@ -9,8 +9,7 @@ package org.opendaylight.controller.cluster.datastore;
 
 import static org.junit.Assert.assertEquals;
 import org.junit.Test;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
 
@@ -20,11 +19,13 @@ import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
  * @author Thomas Pantelis
  */
 public class OperationLimiterTest {
+    private final TransactionIdentifier transactionId = MockIdentifiers.transactionIdentifier(
+        OperationLimiterTest.class, "mock");
 
     @Test
     public void testOnComplete() throws Exception {
         int permits = 10;
-        OperationLimiter limiter = new OperationLimiter(new TransactionIdentifier(MemberName.forName("foo"), 1), permits, 1);
+        OperationLimiter limiter = new OperationLimiter(transactionId, permits, 1);
         limiter.acquire(permits);
         int availablePermits = 0;
 
index ed6a731162d58827ca8452fe8edd5aca2df7061e..d0af812608072fdda4c66057eb632b2581ae4dce 100644 (file)
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import org.junit.Assert;
 import org.junit.Test;
+import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.DefaultShardStrategy;
@@ -41,52 +42,49 @@ import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import scala.concurrent.Promise;
 
 public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
+    private LocalHistoryIdentifier historyId;
+
+    @Override
+    public void setUp() {
+        super.setUp();
+        historyId = MockIdentifiers.historyIdentifier(TransactionChainProxyTest.class, memberName);
+    }
 
     @SuppressWarnings("resource")
     @Test
-    public void testNewReadOnlyTransaction() throws Exception {
+    public void testNewReadOnlyTransaction() {
 
-        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory).newReadOnlyTransaction();
+        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory, historyId).newReadOnlyTransaction();
         Assert.assertTrue(dst instanceof DOMStoreReadTransaction);
 
     }
 
     @SuppressWarnings("resource")
     @Test
-    public void testNewReadWriteTransaction() throws Exception {
-        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory).newReadWriteTransaction();
+    public void testNewReadWriteTransaction() {
+        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory, historyId).newReadWriteTransaction();
         Assert.assertTrue(dst instanceof DOMStoreReadWriteTransaction);
 
     }
 
     @SuppressWarnings("resource")
     @Test
-    public void testNewWriteOnlyTransaction() throws Exception {
-        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory).newWriteOnlyTransaction();
+    public void testNewWriteOnlyTransaction() {
+        DOMStoreTransaction dst = new TransactionChainProxy(mockComponentFactory, historyId).newWriteOnlyTransaction();
         Assert.assertTrue(dst instanceof DOMStoreWriteTransaction);
 
     }
 
     @Test
-    public void testClose() throws Exception {
-        new TransactionChainProxy(mockComponentFactory).close();
+    public void testClose() {
+        new TransactionChainProxy(mockComponentFactory, historyId).close();
 
         verify(mockActorContext, times(1)).broadcast(any(Function.class));
     }
 
-    @Test
-    public void testTransactionChainsHaveUniqueId() {
-        try (TransactionChainProxy one = new TransactionChainProxy(mockComponentFactory)) {
-            try (TransactionChainProxy two = new TransactionChainProxy(mockComponentFactory)) {
-
-                Assert.assertNotEquals(one.getTransactionChainId(), two.getTransactionChainId());
-            }
-        }
-    }
-
     @Test
     public void testRateLimitingUsedInReadWriteTxCreation() {
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             txChainProxy.newReadWriteTransaction();
 
@@ -96,7 +94,7 @@ public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
 
     @Test
     public void testRateLimitingUsedInWriteOnlyTxCreation() {
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             txChainProxy.newWriteOnlyTransaction();
 
@@ -106,7 +104,7 @@ public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
 
     @Test
     public void testRateLimitingNotUsedInReadOnlyTxCreation() {
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             txChainProxy.newReadOnlyTransaction();
 
@@ -122,7 +120,7 @@ public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
     public void testChainedWriteOnlyTransactions() throws Exception {
         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
 
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             ActorRef txActorRef1 = setupActorContextWithoutInitialCreateTransaction(getSystem());
 
@@ -190,7 +188,7 @@ public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
      */
     @Test
     public void testChainedReadWriteTransactions() throws Exception {
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             ActorRef txActorRef1 = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
@@ -258,12 +256,12 @@ public class TransactionChainProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test(expected = IllegalStateException.class)
-    public void testChainedWriteTransactionsWithPreviousTxNotReady() throws Exception {
+    public void testChainedWriteTransactionsWithPreviousTxNotReady() {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
         expectBatchedModifications(actorRef, 1);
 
-        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory)) {
+        try (TransactionChainProxy txChainProxy = new TransactionChainProxy(mockComponentFactory, historyId)) {
 
             DOMStoreWriteTransaction writeTx1 = txChainProxy.newWriteOnlyTransaction();
 
index 10fee5c6d5258e9440a4a79ede3505897016b161..af6e782ed7a45063f3f5b955f755d9227a29f6d7 100644 (file)
@@ -15,27 +15,23 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 
 public class TransactionContextWrapperTest {
-
-    @Mock
-    TransactionIdentifier identifier;
-
     @Mock
-    ActorContext actorContext;
+    private ActorContext actorContext;
 
     @Mock
-    TransactionContext transactionContext;
+    private TransactionContext transactionContext;
 
-    TransactionContextWrapper transactionContextWrapper;
+    private TransactionContextWrapper transactionContextWrapper;
 
     @Before
     public void setUp(){
         MockitoAnnotations.initMocks(this);
         doReturn(DatastoreContext.newBuilder().build()).when(actorContext).getDatastoreContext();
-        transactionContextWrapper = new TransactionContextWrapper(identifier, actorContext);
+        transactionContextWrapper = new TransactionContextWrapper(MockIdentifiers.transactionIdentifier(
+            TransactionContextWrapperTest.class, "mock"), actorContext);
     }
 
     @Test
index 7d23c0affc6063b2c72ca4cdea839baca0460fc8..eb77ed2c2328d56ed532ac4ef753246e6fe9b5ab 100644 (file)
@@ -755,7 +755,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         Object id = transactionProxy.getIdentifier();
         assertNotNull("getIdentifier returned null", id);
-        assertTrue("Invalid identifier: " + id, id.toString().startsWith(memberName));
+        assertTrue("Invalid identifier: " + id, id.toString().contains(MemberName.forName(memberName).toString()));
     }
 
     @Test
index 0bf4f48cead2308e62482e7f0dc4c233e31e3646..f4c777ee61a290f88761144d4789fde097706c51 100644 (file)
@@ -53,7 +53,7 @@ public class PreBoronTransactionProxyTest extends AbstractTransactionProxyTest {
             public boolean matches(Object argument) {
                 if(ShardTransactionMessages.CreateTransaction.class.equals(argument.getClass())) {
                     CreateTransaction obj = CreateTransaction.fromSerializable(argument);
-                    return obj.getTransactionId().startsWith(memberName) &&
+                    return obj.getTransactionId().startsWith(memberName + ':') &&
                             obj.getTransactionType() == type.ordinal();
                 }
 
@@ -119,7 +119,7 @@ public class PreBoronTransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testClose() throws Exception{
+    public void testClose() {
         ActorRef actorRef = setupPreBoronActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
         expectBatchedModifications(actorRef, 1);
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifierTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/ChainedTransactionIdentifierTest.java
deleted file mode 100644 (file)
index 64f04ad..0000000
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.datastore.identifiers;
-
-import static org.hamcrest.CoreMatchers.startsWith;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import org.junit.Test;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-
-public class ChainedTransactionIdentifierTest {
-
-    @Test
-    public void testToString(){
-        TransactionChainIdentifier chainId = new TransactionChainIdentifier(MemberName.forName("member-1"), 99);
-        ChainedTransactionIdentifier chainedTransactionIdentifier = new ChainedTransactionIdentifier(chainId, 100);
-
-        String txnId = chainedTransactionIdentifier.toString();
-
-        assertTrue(txnId.contains("member-1"));
-        assertTrue(txnId.contains("100"));
-        assertTrue(txnId.contains("99"));
-
-        assertThat(txnId, startsWith("member-1-chn-99-txn-100-"));
-    }
-
-}
\ No newline at end of file
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifierTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/identifiers/TransactionChainIdentifierTest.java
deleted file mode 100644 (file)
index b86c94c..0000000
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-
-package org.opendaylight.controller.cluster.datastore.identifiers;
-
-import static org.hamcrest.CoreMatchers.startsWith;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import org.junit.Test;
-import org.opendaylight.controller.cluster.access.concepts.MemberName;
-
-public class TransactionChainIdentifierTest {
-    private static final MemberName MEMBER_1 = MemberName.forName("member-1");
-
-    @Test
-    public void testToString(){
-        TransactionChainIdentifier transactionChainIdentifier = new TransactionChainIdentifier(MEMBER_1, 99);
-
-        String id = transactionChainIdentifier.toString();
-
-        assertEquals("member-1-chn-99", id);
-    }
-
-    @Test
-    public void testNewTransactionIdentifier(){
-        TransactionChainIdentifier transactionChainIdentifier = new TransactionChainIdentifier(MEMBER_1, 99);
-
-        TransactionIdentifier txId1 = transactionChainIdentifier.newTransactionIdentifier();
-
-        assertThat(txId1.toString(), startsWith("member-1-chn-99-txn-1-"));
-
-        TransactionIdentifier txId2 = transactionChainIdentifier.newTransactionIdentifier();
-
-        assertThat(txId2.toString(), startsWith("member-1-chn-99-txn-2-"));
-    }
-
-}
\ No newline at end of file