From be324821e7ef3dba64375e74b920b7ab513c42e3 Mon Sep 17 00:00:00 2001 From: Basheeruddin Ahmed Date: Fri, 29 Aug 2014 12:08:58 -0700 Subject: [PATCH] Include JMX Counters and resetTransactionCounters added ReadFailedTransactionsCount added AbortTransactionsCount and resetTransactionCounters Note: Could not get the details where InMemoryDatastore counters are -- need to reset those counters as part of separate commit included copyright headers in files that didn't had patch 4: Review comments closed patch 3: removed a comment that is not needed. Change-Id: I3b7b81bd6c28d2fb766947df8607e8c824445bb0 Signed-off-by: Basheeruddin Ahmed --- ...hardContext.java => DatastoreContext.java} | 17 +-- .../datastore/DistributedDataStore.java | 8 +- .../controller/cluster/datastore/Shard.java | 32 ++--- .../cluster/datastore/ShardManager.java | 20 +-- .../datastore/ShardReadTransaction.java | 4 +- .../datastore/ShardReadWriteTransaction.java | 4 +- .../cluster/datastore/ShardTransaction.java | 34 +++--- .../datastore/ShardTransactionChain.java | 29 +++-- .../datastore/ShardWriteTransaction.java | 4 +- .../datastore/ThreePhaseCommitCohort.java | 20 ++- .../cluster/datastore/TransactionProxy.java | 3 +- .../jmx/mbeans/AbstractBaseMBean.java | 2 +- .../jmx/mbeans/shard/ShardMBeanFactory.java | 11 +- .../jmx/mbeans/shard/ShardStats.java | 114 +++++++++++++----- .../jmx/mbeans/shard/ShardStatsMBean.java | 26 ++-- .../datastore/BasicIntegrationTest.java | 4 +- .../cluster/datastore/ShardManagerTest.java | 12 +- .../cluster/datastore/ShardTest.java | 10 +- .../datastore/ShardTransactionChainTest.java | 8 +- .../ShardTransactionFailureTest.java | 30 ++--- .../datastore/ShardTransactionTest.java | 48 ++++---- .../ThreePhaseCommitCohortFailureTest.java | 19 +-- .../jmx/mbeans/shard/ShardStatsTest.java | 69 +++++++++++ 23 files changed, 343 insertions(+), 185 deletions(-) rename opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/{ShardContext.java => DatastoreContext.java} (87%) diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DatastoreContext.java similarity index 87% rename from opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardContext.java rename to opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DatastoreContext.java index 02bff77af8..af8a987c73 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardContext.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DatastoreContext.java @@ -8,30 +8,29 @@ package org.opendaylight.controller.cluster.datastore; -import java.util.concurrent.TimeUnit; - -import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreConfigProperties; import com.google.common.base.Preconditions; - +import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreConfigProperties; import scala.concurrent.duration.Duration; +import java.util.concurrent.TimeUnit; + /** * Contains contextual data for shards. * * @author Thomas Pantelis */ -public class ShardContext { +public class DatastoreContext { private final InMemoryDOMDataStoreConfigProperties dataStoreProperties; private final Duration shardTransactionIdleTimeout; - public ShardContext() { + public DatastoreContext() { this.dataStoreProperties = null; this.shardTransactionIdleTimeout = Duration.create(10, TimeUnit.MINUTES); } - public ShardContext(InMemoryDOMDataStoreConfigProperties dataStoreProperties, - Duration shardTransactionIdleTimeout) { + public DatastoreContext(InMemoryDOMDataStoreConfigProperties dataStoreProperties, + Duration shardTransactionIdleTimeout) { this.dataStoreProperties = Preconditions.checkNotNull(dataStoreProperties); this.shardTransactionIdleTimeout = Preconditions.checkNotNull(shardTransactionIdleTimeout); } @@ -43,4 +42,6 @@ public class ShardContext { public Duration getShardTransactionIdleTimeout() { return shardTransactionIdleTimeout; } + + } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DistributedDataStore.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DistributedDataStore.java index b971648392..202ced9a26 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DistributedDataStore.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/DistributedDataStore.java @@ -46,7 +46,7 @@ public class DistributedDataStore implements DOMStore, SchemaContextListener, Au private static final Logger LOG = LoggerFactory.getLogger(DistributedDataStore.class); private final ActorContext actorContext; - private final ShardContext shardContext; + private final DatastoreContext datastoreContext; public DistributedDataStore(ActorSystem actorSystem, String type, ClusterWrapper cluster, Configuration configuration, DistributedDataStoreProperties dataStoreProperties) { @@ -60,7 +60,7 @@ public class DistributedDataStore implements DOMStore, SchemaContextListener, Au LOG.info("Creating ShardManager : {}", shardManagerId); - shardContext = new ShardContext(InMemoryDOMDataStoreConfigProperties.create( + datastoreContext = new DatastoreContext(InMemoryDOMDataStoreConfigProperties.create( dataStoreProperties.getMaxShardDataChangeExecutorPoolSize(), dataStoreProperties.getMaxShardDataChangeExecutorQueueSize(), dataStoreProperties.getMaxShardDataChangeListenerQueueSize()), @@ -70,13 +70,13 @@ public class DistributedDataStore implements DOMStore, SchemaContextListener, Au actorContext = new ActorContext( actorSystem, actorSystem.actorOf( - ShardManager.props(type, cluster, configuration, shardContext). + ShardManager.props(type, cluster, configuration, datastoreContext). withMailbox(ActorContext.MAILBOX), shardManagerId ), cluster, configuration); } public DistributedDataStore(ActorContext actorContext) { this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null"); - this.shardContext = new ShardContext(); + this.datastoreContext = new DatastoreContext(); } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java index abcde747b9..43a9faa3e4 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/Shard.java @@ -96,16 +96,17 @@ public class Shard extends RaftActor { private final List dataChangeListeners = new ArrayList<>(); - private final ShardContext shardContext; + private final DatastoreContext datastoreContext; + private SchemaContext schemaContext; private Shard(ShardIdentifier name, Map peerAddresses, - ShardContext shardContext) { + DatastoreContext datastoreContext) { super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams)); this.name = name; - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; String setting = System.getProperty("shard.persistent"); @@ -114,10 +115,11 @@ public class Shard extends RaftActor { LOG.info("Shard created : {} persistent : {}", name, persistent); store = InMemoryDOMDataStoreFactory.create(name.toString(), null, - shardContext.getDataStoreProperties()); + datastoreContext.getDataStoreProperties()); shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString()); + } private static Map mapPeerAddresses( @@ -134,12 +136,12 @@ public class Shard extends RaftActor { public static Props props(final ShardIdentifier name, final Map peerAddresses, - ShardContext shardContext) { + DatastoreContext datastoreContext) { Preconditions.checkNotNull(name, "name should not be null"); Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null"); - Preconditions.checkNotNull(shardContext, "shardContext should not be null"); + Preconditions.checkNotNull(datastoreContext, "shardContext should not be null"); - return Props.create(new ShardCreator(name, peerAddresses, shardContext)); + return Props.create(new ShardCreator(name, peerAddresses, datastoreContext)); } @Override public void onReceiveCommand(Object message) { @@ -185,7 +187,7 @@ public class Shard extends RaftActor { return getContext().actorOf( ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(), - schemaContext, shardContext), transactionId.toString()); + schemaContext,datastoreContext, name.toString()), transactionId.toString()); } else if (createTransaction.getTransactionType() == TransactionProxy.TransactionType.READ_WRITE.ordinal()) { @@ -194,7 +196,7 @@ public class Shard extends RaftActor { return getContext().actorOf( ShardTransaction.props(store.newReadWriteTransaction(), getSelf(), - schemaContext, shardContext), transactionId.toString()); + schemaContext, datastoreContext,name.toString()), transactionId.toString()); } else if (createTransaction.getTransactionType() @@ -204,7 +206,7 @@ public class Shard extends RaftActor { return getContext().actorOf( ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(), - schemaContext, shardContext), transactionId.toString()); + schemaContext, datastoreContext, name.toString()), transactionId.toString()); } else { throw new IllegalArgumentException( "Shard="+name + ":CreateTransaction message has unidentified transaction type=" @@ -343,7 +345,7 @@ public class Shard extends RaftActor { private void createTransactionChain() { DOMStoreTransactionChain chain = store.createTransactionChain(); ActorRef transactionChain = getContext().actorOf( - ShardTransactionChain.props(chain, schemaContext, shardContext)); + ShardTransactionChain.props(chain, schemaContext, datastoreContext,name.toString() )); getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(), getSelf()); } @@ -423,18 +425,18 @@ public class Shard extends RaftActor { final ShardIdentifier name; final Map peerAddresses; - final ShardContext shardContext; + final DatastoreContext datastoreContext; ShardCreator(ShardIdentifier name, Map peerAddresses, - ShardContext shardContext) { + DatastoreContext datastoreContext) { this.name = name; this.peerAddresses = peerAddresses; - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; } @Override public Shard create() throws Exception { - return new Shard(name, peerAddresses, shardContext); + return new Shard(name, peerAddresses, datastoreContext); } } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java index 186f2cff41..e51d49bff2 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardManager.java @@ -72,19 +72,19 @@ public class ShardManager extends AbstractUntypedActor { private ShardManagerInfoMBean mBean; - private final ShardContext shardContext; + private final DatastoreContext datastoreContext; /** * @param type defines the kind of data that goes into shards created by this shard manager. Examples of type would be * configuration or operational */ private ShardManager(String type, ClusterWrapper cluster, Configuration configuration, - ShardContext shardContext) { + DatastoreContext datastoreContext) { this.type = Preconditions.checkNotNull(type, "type should not be null"); this.cluster = Preconditions.checkNotNull(cluster, "cluster should not be null"); this.configuration = Preconditions.checkNotNull(configuration, "configuration should not be null"); - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; // Subscribe this actor to cluster member events cluster.subscribeToMemberEvents(getSelf()); @@ -97,13 +97,13 @@ public class ShardManager extends AbstractUntypedActor { public static Props props(final String type, final ClusterWrapper cluster, final Configuration configuration, - final ShardContext shardContext) { + final DatastoreContext datastoreContext) { Preconditions.checkNotNull(type, "type should not be null"); Preconditions.checkNotNull(cluster, "cluster should not be null"); Preconditions.checkNotNull(configuration, "configuration should not be null"); - return Props.create(new ShardManagerCreator(type, cluster, configuration, shardContext)); + return Props.create(new ShardManagerCreator(type, cluster, configuration, datastoreContext)); } @Override @@ -245,7 +245,7 @@ public class ShardManager extends AbstractUntypedActor { ShardIdentifier shardId = getShardIdentifier(memberName, shardName); Map peerAddresses = getPeerAddresses(shardName); ActorRef actor = getContext() - .actorOf(Shard.props(shardId, peerAddresses, shardContext). + .actorOf(Shard.props(shardId, peerAddresses, datastoreContext). withMailbox(ActorContext.MAILBOX), shardId.toString()); localShardActorNames.add(shardId.toString()); @@ -354,19 +354,19 @@ public class ShardManager extends AbstractUntypedActor { final String type; final ClusterWrapper cluster; final Configuration configuration; - final ShardContext shardContext; + final DatastoreContext datastoreContext; ShardManagerCreator(String type, ClusterWrapper cluster, - Configuration configuration, ShardContext shardContext) { + Configuration configuration, DatastoreContext datastoreContext) { this.type = type; this.cluster = cluster; this.configuration = configuration; - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; } @Override public ShardManager create() throws Exception { - return new ShardManager(type, cluster, configuration, shardContext); + return new ShardManager(type, cluster, configuration, datastoreContext); } } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadTransaction.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadTransaction.java index c54d3739b2..91d629432f 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadTransaction.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadTransaction.java @@ -26,8 +26,8 @@ public class ShardReadTransaction extends ShardTransaction { private final DOMStoreReadTransaction transaction; public ShardReadTransaction(DOMStoreReadTransaction transaction, ActorRef shardActor, - SchemaContext schemaContext) { - super(shardActor, schemaContext); + SchemaContext schemaContext,String shardName) { + super(shardActor, schemaContext, shardName); this.transaction = transaction; } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadWriteTransaction.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadWriteTransaction.java index 1dc9ce0a3e..bd71c27fd6 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadWriteTransaction.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardReadWriteTransaction.java @@ -30,8 +30,8 @@ public class ShardReadWriteTransaction extends ShardTransaction { private final DOMStoreReadWriteTransaction transaction; public ShardReadWriteTransaction(DOMStoreReadWriteTransaction transaction, ActorRef shardActor, - SchemaContext schemaContext) { - super(shardActor, schemaContext); + SchemaContext schemaContext,String shardName) { + super(shardActor, schemaContext, shardName); this.transaction = transaction; } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java index 365960713d..3b0e0934d9 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransaction.java @@ -13,11 +13,10 @@ import akka.actor.PoisonPill; import akka.actor.Props; import akka.actor.ReceiveTimeout; import akka.japi.Creator; - import com.google.common.base.Optional; import com.google.common.util.concurrent.CheckedFuture; - import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException; +import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory; import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction; import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.DataExists; @@ -74,18 +73,22 @@ public abstract class ShardTransaction extends AbstractUntypedActor { private final ActorRef shardActor; protected final SchemaContext schemaContext; + private final String shardName; + private final MutableCompositeModification modification = new MutableCompositeModification(); - protected ShardTransaction(ActorRef shardActor, SchemaContext schemaContext) { + protected ShardTransaction(ActorRef shardActor, SchemaContext schemaContext, + String shardName) { this.shardActor = shardActor; this.schemaContext = schemaContext; + this.shardName = shardName; } public static Props props(DOMStoreTransaction transaction, ActorRef shardActor, - SchemaContext schemaContext, ShardContext shardContext) { + SchemaContext schemaContext,DatastoreContext datastoreContext, String shardName) { return Props.create(new ShardTransactionCreator(transaction, shardActor, schemaContext, - shardContext)); + datastoreContext, shardName)); } protected abstract DOMStoreTransaction getDOMStoreTransaction(); @@ -134,7 +137,8 @@ public abstract class ShardTransaction extends AbstractUntypedActor { sender.tell(new ReadDataReply(schemaContext,null).toSerializable(), self); } } catch (Exception e) { - sender.tell(new akka.actor.Status.Failure(e),self); + ShardMBeanFactory.getShardStatsMBean(shardName).incrementFailedReadTransactionsCount(); + sender.tell(new akka.actor.Status.Failure(e), self); } } @@ -192,7 +196,7 @@ public abstract class ShardTransaction extends AbstractUntypedActor { protected void readyTransaction(DOMStoreWriteTransaction transaction, ReadyTransaction message) { DOMStoreThreePhaseCommitCohort cohort = transaction.ready(); ActorRef cohortActor = getContext().actorOf( - ThreePhaseCommitCohort.props(cohort, shardActor, modification), "cohort"); + ThreePhaseCommitCohort.props(cohort, shardActor, modification, shardName), "cohort"); getSender() .tell(new ReadyTransactionReply(cohortActor.path()).toSerializable(), getSelf()); @@ -205,14 +209,16 @@ public abstract class ShardTransaction extends AbstractUntypedActor { final DOMStoreTransaction transaction; final ActorRef shardActor; final SchemaContext schemaContext; - final ShardContext shardContext; + final DatastoreContext datastoreContext; + final String shardName; ShardTransactionCreator(DOMStoreTransaction transaction, ActorRef shardActor, - SchemaContext schemaContext, ShardContext actorContext) { + SchemaContext schemaContext, DatastoreContext datastoreContext, String shardName) { this.transaction = transaction; this.shardActor = shardActor; - this.shardContext = actorContext; + this.shardName = shardName; this.schemaContext = schemaContext; + this.datastoreContext = datastoreContext; } @Override @@ -220,16 +226,16 @@ public abstract class ShardTransaction extends AbstractUntypedActor { ShardTransaction tx; if(transaction instanceof DOMStoreReadWriteTransaction) { tx = new ShardReadWriteTransaction((DOMStoreReadWriteTransaction)transaction, - shardActor, schemaContext); + shardActor, schemaContext, shardName); } else if(transaction instanceof DOMStoreReadTransaction) { tx = new ShardReadTransaction((DOMStoreReadTransaction)transaction, shardActor, - schemaContext); + schemaContext, shardName); } else { tx = new ShardWriteTransaction((DOMStoreWriteTransaction)transaction, - shardActor, schemaContext); + shardActor, schemaContext, shardName); } - tx.getContext().setReceiveTimeout(shardContext.getShardTransactionIdleTimeout()); + tx.getContext().setReceiveTimeout(datastoreContext.getShardTransactionIdleTimeout()); return tx; } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChain.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChain.java index 42bd257ad1..e7a181865e 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChain.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChain.java @@ -25,14 +25,16 @@ import org.opendaylight.yangtools.yang.model.api.SchemaContext; public class ShardTransactionChain extends AbstractUntypedActor { private final DOMStoreTransactionChain chain; - private final ShardContext shardContext; + private final DatastoreContext datastoreContext; private final SchemaContext schemaContext; + private final String shardName; public ShardTransactionChain(DOMStoreTransactionChain chain, SchemaContext schemaContext, - ShardContext shardContext) { + DatastoreContext datastoreContext,String shardName) { this.chain = chain; - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; this.schemaContext = schemaContext; + this.shardName = shardName; } @Override @@ -58,17 +60,17 @@ public class ShardTransactionChain extends AbstractUntypedActor { TransactionProxy.TransactionType.READ_ONLY.ordinal()) { return getContext().actorOf( ShardTransaction.props( chain.newReadOnlyTransaction(), getShardActor(), - schemaContext, shardContext), transactionId); + schemaContext, datastoreContext,shardName), transactionId); } else if (createTransaction.getTransactionType() == TransactionProxy.TransactionType.READ_WRITE.ordinal()) { return getContext().actorOf( ShardTransaction.props( chain.newReadWriteTransaction(), getShardActor(), - schemaContext, shardContext), transactionId); + schemaContext, datastoreContext,shardName), transactionId); } else if (createTransaction.getTransactionType() == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) { return getContext().actorOf( ShardTransaction.props( chain.newWriteOnlyTransaction(), getShardActor(), - schemaContext, shardContext), transactionId); + schemaContext, datastoreContext,shardName), transactionId); } else { throw new IllegalArgumentException ( "CreateTransaction message has unidentified transaction type=" + @@ -85,27 +87,30 @@ public class ShardTransactionChain extends AbstractUntypedActor { } public static Props props(DOMStoreTransactionChain chain, SchemaContext schemaContext, - ShardContext shardContext) { - return Props.create(new ShardTransactionChainCreator(chain, schemaContext, shardContext)); + DatastoreContext datastoreContext, String shardName) { + return Props.create(new ShardTransactionChainCreator(chain, schemaContext, datastoreContext, shardName)); } private static class ShardTransactionChainCreator implements Creator { private static final long serialVersionUID = 1L; final DOMStoreTransactionChain chain; - final ShardContext shardContext; + final DatastoreContext datastoreContext; final SchemaContext schemaContext; + final String shardName; + ShardTransactionChainCreator(DOMStoreTransactionChain chain, SchemaContext schemaContext, - ShardContext shardContext) { + DatastoreContext datastoreContext, String shardName) { this.chain = chain; - this.shardContext = shardContext; + this.datastoreContext = datastoreContext; this.schemaContext = schemaContext; + this.shardName = shardName; } @Override public ShardTransactionChain create() throws Exception { - return new ShardTransactionChain(chain, schemaContext, shardContext); + return new ShardTransactionChain(chain, schemaContext, datastoreContext,shardName); } } } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardWriteTransaction.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardWriteTransaction.java index 8b4d576163..41c46c3375 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardWriteTransaction.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardWriteTransaction.java @@ -28,8 +28,8 @@ public class ShardWriteTransaction extends ShardTransaction { private final DOMStoreWriteTransaction transaction; public ShardWriteTransaction(DOMStoreWriteTransaction transaction, ActorRef shardActor, - SchemaContext schemaContext) { - super(shardActor, schemaContext); + SchemaContext schemaContext,String shardName) { + super(shardActor, schemaContext, shardName); this.transaction = transaction; } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohort.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohort.java index d0c29294cb..5a6d0eca5c 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohort.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohort.java @@ -19,6 +19,7 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory; import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction; import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply; import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction; @@ -34,21 +35,25 @@ public class ThreePhaseCommitCohort extends AbstractUntypedActor { private final DOMStoreThreePhaseCommitCohort cohort; private final ActorRef shardActor; private final CompositeModification modification; + private final String shardName; public ThreePhaseCommitCohort(DOMStoreThreePhaseCommitCohort cohort, - ActorRef shardActor, CompositeModification modification) { + ActorRef shardActor, CompositeModification modification,String shardName) { this.cohort = cohort; this.shardActor = shardActor; this.modification = modification; + this.shardName = shardName; } private final LoggingAdapter log = Logging.getLogger(getContext().system(), this); public static Props props(final DOMStoreThreePhaseCommitCohort cohort, - final ActorRef shardActor, final CompositeModification modification) { - return Props.create(new ThreePhaseCommitCohortCreator(cohort, shardActor, modification)); + final ActorRef shardActor, final CompositeModification modification, + String shardName) { + return Props.create(new ThreePhaseCommitCohortCreator(cohort, shardActor, modification, + shardName)); } @Override @@ -78,9 +83,10 @@ public class ThreePhaseCommitCohort extends AbstractUntypedActor { Futures.addCallback(future, new FutureCallback() { @Override public void onSuccess(Void v) { + ShardMBeanFactory.getShardStatsMBean(shardName).incrementAbortTransactionsCount(); sender .tell(new AbortTransactionReply().toSerializable(), - self); + self); } @Override @@ -148,17 +154,19 @@ public class ThreePhaseCommitCohort extends AbstractUntypedActor { final DOMStoreThreePhaseCommitCohort cohort; final ActorRef shardActor; final CompositeModification modification; + final String shardName; ThreePhaseCommitCohortCreator(DOMStoreThreePhaseCommitCohort cohort, - ActorRef shardActor, CompositeModification modification) { + ActorRef shardActor, CompositeModification modification, String shardName) { this.cohort = cohort; this.shardActor = shardActor; this.modification = modification; + this.shardName = shardName; } @Override public ThreePhaseCommitCohort create() throws Exception { - return new ThreePhaseCommitCohort(cohort, shardActor, modification); + return new ThreePhaseCommitCohort(cohort, shardActor, modification, shardName); } } } 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 f447f3c718..fc1a3aad74 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 @@ -249,7 +249,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction { Preconditions.checkState(transactionType != TransactionType.READ_ONLY, "Modification operation on read-only transaction is not allowed"); Preconditions.checkState(!inReadyState, - "Transaction is sealed - further modifications are allowed"); + "Transaction is sealed - further modifications are not allowed"); } @Override @@ -612,6 +612,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction { returnFuture.setException(new ReadFailedException( "Error reading data for path " + path, failure)); + } else { LOG.debug("Tx {} read operation succeeded", identifier, failure); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/AbstractBaseMBean.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/AbstractBaseMBean.java index a5d7b77a64..3c46935d98 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/AbstractBaseMBean.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/AbstractBaseMBean.java @@ -24,7 +24,7 @@ import java.lang.management.ManagementFactory; /** * All MBeans should extend this class that help in registering and * unregistering the MBeans. - * + * @author Basheeruddin */ diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardMBeanFactory.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardMBeanFactory.java index afca87f0df..2a409c0300 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardMBeanFactory.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardMBeanFactory.java @@ -1,11 +1,18 @@ +/* + * 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.jmx.mbeans.shard; import java.util.HashMap; import java.util.Map; /** - * @author: syedbahm - * Date: 7/16/14 + * @author Basheeruddin syedbahm@cisco.com + * */ public class ShardMBeanFactory { private static Map shardMBeans = diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStats.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStats.java index c6c1579ce3..22ad8e7f5a 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStats.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStats.java @@ -1,3 +1,11 @@ +/* + * 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.jmx.mbeans.shard; import org.opendaylight.controller.cluster.datastore.jmx.mbeans.AbstractBaseMBean; @@ -6,37 +14,41 @@ import java.text.SimpleDateFormat; import java.util.Date; /** - * @author: syedbahm + * @author Basheeruddin syedbahm@cisco.com */ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { private final String shardName; - private Long committedTransactionsCount = 0L; + private long committedTransactionsCount = 0L; - private Long readOnlyTransactionCount = 0L; + private long readOnlyTransactionCount = 0L; - private Long writeOnlyTransactionCount = 0L; + private long writeOnlyTransactionCount = 0L; - private Long readWriteTransactionCount = 0L; + private long readWriteTransactionCount = 0L; private String leader; private String raftState; - private Long lastLogTerm = -1L; + private long lastLogTerm = -1L; - private Long lastLogIndex = -1L; + private long lastLogIndex = -1L; - private Long currentTerm = -1L; + private long currentTerm = -1L; - private Long commitIndex = -1L; + private long commitIndex = -1L; - private Long lastApplied = -1L; + private long lastApplied = -1L; private Date lastCommittedTransactionTime = new Date(0L); - private Long failedTransactionsCount = 0L; + private long failedTransactionsCount = 0L; + + private long failedReadTransactionsCount = 0L; + + private long abortTransactionsCount = 0L; private SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); @@ -52,7 +64,7 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { } @Override - public Long getCommittedTransactionsCount() { + public long getCommittedTransactionsCount() { return committedTransactionsCount; } @@ -64,35 +76,35 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { return raftState; } - @Override public Long getReadOnlyTransactionCount() { + @Override public long getReadOnlyTransactionCount() { return readOnlyTransactionCount; } - @Override public Long getWriteOnlyTransactionCount() { + @Override public long getWriteOnlyTransactionCount() { return writeOnlyTransactionCount; } - @Override public Long getReadWriteTransactionCount() { + @Override public long getReadWriteTransactionCount() { return readWriteTransactionCount; } - @Override public Long getLastLogIndex() { + @Override public long getLastLogIndex() { return lastLogIndex; } - @Override public Long getLastLogTerm() { + @Override public long getLastLogTerm() { return lastLogTerm; } - @Override public Long getCurrentTerm() { + @Override public long getCurrentTerm() { return currentTerm; } - @Override public Long getCommitIndex() { + @Override public long getCommitIndex() { return commitIndex; } - @Override public Long getLastApplied() { + @Override public long getLastApplied() { return lastApplied; } @@ -102,26 +114,44 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { return sdf.format(lastCommittedTransactionTime); } - @Override public Long getFailedTransactionsCount() { + @Override public long getFailedTransactionsCount() { return failedTransactionsCount; } - public Long incrementCommittedTransactionCount() { + @Override public long getFailedReadTransactionsCount() { + return failedReadTransactionsCount; + } + + @Override public long getAbortTransactionsCount() { + return abortTransactionsCount; + } + + public long incrementCommittedTransactionCount() { return committedTransactionsCount++; } - public Long incrementReadOnlyTransactionCount() { + public long incrementReadOnlyTransactionCount() { return readOnlyTransactionCount++; } - public Long incrementWriteOnlyTransactionCount() { + public long incrementWriteOnlyTransactionCount() { return writeOnlyTransactionCount++; } - public Long incrementReadWriteTransactionCount() { + public long incrementReadWriteTransactionCount() { return readWriteTransactionCount++; } + public long incrementFailedTransactionsCount() { + return failedTransactionsCount++; + } + + public long incrementFailedReadTransactionsCount() { + return failedReadTransactionsCount++; + } + + public long incrementAbortTransactionsCount () { return abortTransactionsCount++;} + public void setLeader(String leader) { this.leader = leader; } @@ -130,23 +160,23 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { this.raftState = raftState; } - public void setLastLogTerm(Long lastLogTerm) { + public void setLastLogTerm(long lastLogTerm) { this.lastLogTerm = lastLogTerm; } - public void setLastLogIndex(Long lastLogIndex) { + public void setLastLogIndex(long lastLogIndex) { this.lastLogIndex = lastLogIndex; } - public void setCurrentTerm(Long currentTerm) { + public void setCurrentTerm(long currentTerm) { this.currentTerm = currentTerm; } - public void setCommitIndex(Long commitIndex) { + public void setCommitIndex(long commitIndex) { this.commitIndex = commitIndex; } - public void setLastApplied(Long lastApplied) { + public void setLastApplied(long lastApplied) { this.lastApplied = lastApplied; } @@ -171,8 +201,28 @@ public class ShardStats extends AbstractBaseMBean implements ShardStatsMBean { return JMX_CATEGORY_SHARD; } + /** + * resets the counters related to transactions + */ + + public void resetTransactionCounters(){ + committedTransactionsCount = 0L; + + readOnlyTransactionCount = 0L; + + writeOnlyTransactionCount = 0L; + + readWriteTransactionCount = 0L; + + lastCommittedTransactionTime = new Date(0L); + + failedTransactionsCount = 0L; + + failedReadTransactionsCount = 0L; + + abortTransactionsCount = 0L; - public void incrementFailedTransactionsCount() { - this.failedTransactionsCount++; } + + } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsMBean.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsMBean.java index b8b220ee82..c16f8421bf 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsMBean.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsMBean.java @@ -6,30 +6,36 @@ package org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard; public interface ShardStatsMBean { String getShardName(); - Long getCommittedTransactionsCount(); + long getCommittedTransactionsCount(); String getLeader(); String getRaftState(); - Long getReadOnlyTransactionCount(); + long getReadOnlyTransactionCount(); - Long getWriteOnlyTransactionCount(); + long getWriteOnlyTransactionCount(); - Long getReadWriteTransactionCount(); + long getReadWriteTransactionCount(); - Long getLastLogIndex(); + long getLastLogIndex(); - Long getLastLogTerm(); + long getLastLogTerm(); - Long getCurrentTerm(); + long getCurrentTerm(); - Long getCommitIndex(); + long getCommitIndex(); - Long getLastApplied(); + long getLastApplied(); String getLastCommittedTransactionTime(); - Long getFailedTransactionsCount(); + long getFailedTransactionsCount(); + + long getFailedReadTransactionsCount(); + + long getAbortTransactionsCount(); + + void resetTransactionCounters(); } diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java index e70453f2d6..6f131f301f 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/BasicIntegrationTest.java @@ -62,9 +62,9 @@ public class BasicIntegrationTest extends AbstractActorTest { .shardName("inventory").type("config").build(); final SchemaContext schemaContext = TestModel.createTestContext(); - ShardContext shardContext = new ShardContext(); + DatastoreContext datastoreContext = new DatastoreContext(); - final Props props = Shard.props(identifier, Collections.EMPTY_MAP, shardContext); + final Props props = Shard.props(identifier, Collections.EMPTY_MAP, datastoreContext); final ActorRef shard = getSystem().actorOf(props); new Within(duration("10 seconds")) { diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardManagerTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardManagerTest.java index d16e4f53de..1feefd1c1f 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardManagerTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardManagerTest.java @@ -42,7 +42,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", new MockClusterWrapper(), - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); @@ -67,7 +67,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", new MockClusterWrapper(), - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); @@ -91,7 +91,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", new MockClusterWrapper(), - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); @@ -128,7 +128,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", mockClusterWrapper, - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); @@ -164,7 +164,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", new MockClusterWrapper(), - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); @@ -204,7 +204,7 @@ public class ShardManagerTest { new JavaTestKit(system) {{ final Props props = ShardManager .props("config", new MockClusterWrapper(), - new MockConfiguration(), new ShardContext()); + new MockConfiguration(), new DatastoreContext()); final TestActorRef subject = TestActorRef.create(system, props); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java index fc45efcdea..4466e50f96 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTest.java @@ -35,7 +35,7 @@ import static org.junit.Assert.assertTrue; public class ShardTest extends AbstractActorTest { - private static final ShardContext shardContext = new ShardContext(); + private static final DatastoreContext DATA_STORE_CONTEXT = new DatastoreContext(); @Test public void testOnReceiveCreateTransactionChain() throws Exception { @@ -44,7 +44,7 @@ public class ShardTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("config").build(); - final Props props = Shard.props(identifier, Collections.EMPTY_MAP, shardContext); + final Props props = Shard.props(identifier, Collections.EMPTY_MAP, DATA_STORE_CONTEXT); final ActorRef subject = getSystem().actorOf(props, "testCreateTransactionChain"); @@ -103,7 +103,7 @@ public class ShardTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("config").build(); - final Props props = Shard.props(identifier, Collections.EMPTY_MAP, shardContext); + final Props props = Shard.props(identifier, Collections.EMPTY_MAP, DATA_STORE_CONTEXT); final ActorRef subject = getSystem().actorOf(props, "testRegisterChangeListener"); @@ -165,7 +165,7 @@ public class ShardTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("config").build(); - final Props props = Shard.props(identifier, Collections.EMPTY_MAP, shardContext); + final Props props = Shard.props(identifier, Collections.EMPTY_MAP, DATA_STORE_CONTEXT); final ActorRef subject = getSystem().actorOf(props, "testCreateTransaction"); @@ -227,7 +227,7 @@ public class ShardTest extends AbstractActorTest { .shardName("inventory").type("config").build(); peerAddresses.put(identifier, null); - final Props props = Shard.props(identifier, peerAddresses, shardContext); + final Props props = Shard.props(identifier, peerAddresses, DATA_STORE_CONTEXT); final ActorRef subject = getSystem().actorOf(props, "testPeerAddressResolved"); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChainTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChainTest.java index 0262b8c145..71eb1f1603 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChainTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionChainTest.java @@ -28,7 +28,9 @@ public class ShardTransactionChainTest extends AbstractActorTest { private static final SchemaContext testSchemaContext = TestModel.createTestContext(); - private static final ShardContext shardContext = new ShardContext(); + private static final DatastoreContext DATA_STORE_CONTEXT = new DatastoreContext(); + + private static final String mockShardName = "mockShardName"; @BeforeClass public static void staticSetup() { @@ -39,7 +41,7 @@ public class ShardTransactionChainTest extends AbstractActorTest { public void testOnReceiveCreateTransaction() throws Exception { new JavaTestKit(getSystem()) {{ final Props props = ShardTransactionChain.props(store.createTransactionChain(), - testSchemaContext, shardContext); + testSchemaContext, DATA_STORE_CONTEXT, mockShardName); final ActorRef subject = getSystem().actorOf(props, "testCreateTransaction"); new Within(duration("1 seconds")) { @@ -77,7 +79,7 @@ public class ShardTransactionChainTest extends AbstractActorTest { public void testOnReceiveCloseTransactionChain() throws Exception { new JavaTestKit(getSystem()) {{ final Props props = ShardTransactionChain.props(store.createTransactionChain(), - testSchemaContext, shardContext); + testSchemaContext, DATA_STORE_CONTEXT,mockShardName ); final ActorRef subject = getSystem().actorOf(props, "testCloseTransactionChain"); new Within(duration("1 seconds")) { diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java index 8a6534b87e..4fe60f6467 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionFailureTest.java @@ -57,7 +57,7 @@ public class ShardTransactionFailureTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("operational").build(); - private final ShardContext shardContext = new ShardContext(); + private final DatastoreContext datastoreContext = new DatastoreContext(); @BeforeClass public static void staticSetup() { @@ -69,9 +69,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { throws Throwable { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -99,9 +99,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { throws Throwable { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -129,9 +129,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { throws Throwable { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -159,9 +159,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newWriteOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -192,9 +192,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -230,9 +230,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, "testNegativeMergeTransactionReady"); @@ -263,9 +263,9 @@ public class ShardTransactionFailureTest extends AbstractActorTest { final ActorRef shard = - getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new ShardContext())); + getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java index 1dd824568a..ff2ee08f94 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ShardTransactionTest.java @@ -60,7 +60,7 @@ public class ShardTransactionTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("config").build(); - private ShardContext shardContext = new ShardContext(); + private DatastoreContext datastoreContext = new DatastoreContext(); @BeforeClass public static void staticSetup() { @@ -71,9 +71,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveReadData() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testReadData"); new Within(duration("1 seconds")) { @@ -114,9 +114,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveReadDataWhenDataNotFound() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props( store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testReadDataWhenDataNotFound"); new Within(duration("1 seconds")) { @@ -158,9 +158,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveDataExistsPositive() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testDataExistsPositive"); new Within(duration("1 seconds")) { @@ -201,9 +201,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveDataExistsNegative() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testDataExistsNegative"); new Within(duration("1 seconds")) { @@ -279,9 +279,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveWriteData() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newWriteOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testWriteData"); @@ -320,9 +320,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveMergeData() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testMergeData"); @@ -362,9 +362,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveDeleteData() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props( store.newWriteOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testDeleteData"); @@ -402,9 +402,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveReadyTransaction() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props( store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testReadyTransaction"); @@ -441,9 +441,9 @@ public class ShardTransactionTest extends AbstractActorTest { public void testOnReceiveCloseTransaction() throws Exception { new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testCloseTransaction"); @@ -492,9 +492,9 @@ public class ShardTransactionTest extends AbstractActorTest { @Test(expected=UnknownMessageException.class) public void testNegativePerformingWriteOperationOnReadTransaction() throws Exception { final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef.apply(props,getSystem()); subject.receive(new DeleteData(TestModel.TEST_PATH).toSerializable(), ActorRef.noSender()); @@ -503,14 +503,14 @@ public class ShardTransactionTest extends AbstractActorTest { @Test public void testShardTransactionInactivity() { - shardContext = new ShardContext(InMemoryDOMDataStoreConfigProperties.getDefault(), + datastoreContext = new DatastoreContext(InMemoryDOMDataStoreConfigProperties.getDefault(), Duration.create(500, TimeUnit.MILLISECONDS)); new JavaTestKit(getSystem()) {{ final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, new ShardContext())); + Collections.EMPTY_MAP, new DatastoreContext())); final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard, - testSchemaContext, shardContext); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString()); final ActorRef subject = getSystem().actorOf(props, "testShardTransactionInactivity"); diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortFailureTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortFailureTest.java index 672166c442..e10570cd15 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortFailureTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/ThreePhaseCommitCohortFailureTest.java @@ -63,7 +63,8 @@ public class ThreePhaseCommitCohortFailureTest extends AbstractActorTest { ShardIdentifier.builder().memberName("member-1") .shardName("inventory").type("config").build(); - private final ShardContext shardContext = new ShardContext(); + private final DatastoreContext datastoreContext = new DatastoreContext(); + @BeforeClass public static void staticSetup() { @@ -77,13 +78,13 @@ public class ThreePhaseCommitCohortFailureTest extends AbstractActorTest { public void testNegativeAbortResultsInException() throws Exception { final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, shardContext)); + Collections.EMPTY_MAP, datastoreContext)); final DOMStoreThreePhaseCommitCohort mockCohort = Mockito .mock(DOMStoreThreePhaseCommitCohort.class); final CompositeModification mockComposite = Mockito.mock(CompositeModification.class); final Props props = - ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite); + ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -106,13 +107,13 @@ public class ThreePhaseCommitCohortFailureTest extends AbstractActorTest { public void testNegativeCanCommitResultsInException() throws Exception { final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, shardContext)); + Collections.EMPTY_MAP, datastoreContext)); final DOMStoreThreePhaseCommitCohort mockCohort = Mockito .mock(DOMStoreThreePhaseCommitCohort.class); final CompositeModification mockComposite = Mockito.mock(CompositeModification.class); final Props props = - ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite); + ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -138,13 +139,13 @@ public class ThreePhaseCommitCohortFailureTest extends AbstractActorTest { public void testNegativePreCommitResultsInException() throws Exception { final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER, - Collections.EMPTY_MAP, shardContext)); + Collections.EMPTY_MAP, datastoreContext)); final DOMStoreThreePhaseCommitCohort mockCohort = Mockito .mock(DOMStoreThreePhaseCommitCohort.class); final CompositeModification mockComposite = Mockito.mock(CompositeModification.class); final Props props = - ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite); + ThreePhaseCommitCohort.props(mockCohort, shard, mockComposite,SHARD_IDENTIFIER.toString()); final TestActorRef subject = TestActorRef .create(getSystem(), props, @@ -168,12 +169,12 @@ public class ThreePhaseCommitCohortFailureTest extends AbstractActorTest { public void testNegativeCommitResultsInException() throws Exception { final TestActorRef subject = TestActorRef.create(getSystem(), - Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, shardContext), + Shard.props(SHARD_IDENTIFIER, Collections.EMPTY_MAP, datastoreContext), "testNegativeCommitResultsInException"); final ActorRef shardTransaction = getSystem().actorOf(ShardTransaction.props(store.newReadWriteTransaction(), subject, - testSchemaContext, shardContext)); + testSchemaContext, datastoreContext,SHARD_IDENTIFIER.toString())); ShardTransactionMessages.WriteData writeData = ShardTransactionMessages.WriteData.newBuilder() diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsTest.java b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsTest.java index 41adcc55b1..c4d0b85fb5 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsTest.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/test/java/org/opendaylight/controller/cluster/datastore/jmx/mbeans/shard/ShardStatsTest.java @@ -1,3 +1,10 @@ +/* + * 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.jmx.mbeans.shard; import org.junit.After; @@ -86,7 +93,69 @@ public class ShardStatsTest { Object attribute = mbeanServer.getAttribute(testMBeanName, "FailedTransactionsCount"); Assert.assertEquals((Long) attribute, (Long) 2L); + } + + @Test + public void testGetAbortTransactionsCount() throws Exception { + //let us increment AbortTransactions count and then check + shardStats.incrementAbortTransactionsCount(); + shardStats.incrementAbortTransactionsCount(); + + + //now let us get from MBeanServer what is the transaction count. + Object attribute = + mbeanServer.getAttribute(testMBeanName, "AbortTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 2L); + } + + @Test + public void testGetFailedReadTransactionsCount() throws Exception { + //let us increment FailedReadTransactions count and then check + shardStats.incrementFailedReadTransactionsCount(); + shardStats.incrementFailedReadTransactionsCount(); + + + //now let us get from MBeanServer what is the transaction count. + Object attribute = + mbeanServer.getAttribute(testMBeanName, "FailedReadTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 2L); + } + + @Test + public void testResetTransactionCounters() throws Exception { + + //let us increment committed transactions count and then check + shardStats.incrementCommittedTransactionCount(); + shardStats.incrementCommittedTransactionCount(); + shardStats.incrementCommittedTransactionCount(); + + //now let us get from MBeanServer what is the transaction count. + Object attribute = mbeanServer.getAttribute(testMBeanName, + "CommittedTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 3L); + + //let us increment FailedReadTransactions count and then check + shardStats.incrementFailedReadTransactionsCount(); + shardStats.incrementFailedReadTransactionsCount(); + + + //now let us get from MBeanServer what is the transaction count. + attribute = + mbeanServer.getAttribute(testMBeanName, "FailedReadTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 2L); + + + //here we will reset the counters and check the above ones are 0 after reset + mbeanServer.invoke(testMBeanName, "resetTransactionCounters", null, null); + + //now let us get from MBeanServer what is the transaction count. + attribute = mbeanServer.getAttribute(testMBeanName, + "CommittedTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 0L); + attribute = + mbeanServer.getAttribute(testMBeanName, "FailedReadTransactionsCount"); + Assert.assertEquals((Long) attribute, (Long) 0L); } -- 2.36.6