Include JMX Counters and resetTransactionCounters
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
index 75f540ade088e6bb45b10e9b92bc4af8789b0218..43a9faa3e44e5d6fe77db7cce74929f8888dcadb 100644 (file)
@@ -15,11 +15,13 @@ import akka.event.Logging;
 import akka.event.LoggingAdapter;
 import akka.japi.Creator;
 import akka.serialization.Serialization;
+
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
@@ -43,14 +45,15 @@ import org.opendaylight.controller.cluster.raft.RaftActor;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
-import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreConfigProperties;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
+import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+
 import scala.concurrent.duration.FiniteDuration;
 
 import java.util.ArrayList;
@@ -89,17 +92,21 @@ public class Shard extends RaftActor {
     /// The name of this shard
     private final ShardIdentifier name;
 
-    private volatile SchemaContext schemaContext;
-
     private final ShardStats shardMBean;
 
     private final List<ActorSelection> dataChangeListeners = new ArrayList<>();
 
+    private final DatastoreContext datastoreContext;
+
+
+    private SchemaContext schemaContext;
+
     private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
-            InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
+            DatastoreContext datastoreContext) {
         super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
 
         this.name = name;
+        this.datastoreContext = datastoreContext;
 
         String setting = System.getProperty("shard.persistent");
 
@@ -107,10 +114,12 @@ public class Shard extends RaftActor {
 
         LOG.info("Shard created : {} persistent : {}", name, persistent);
 
-        store = InMemoryDOMDataStoreFactory.create(name.toString(), null, dataStoreProperties);
+        store = InMemoryDOMDataStoreFactory.create(name.toString(), null,
+                datastoreContext.getDataStoreProperties());
 
         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString());
 
+
     }
 
     private static Map<String, String> mapPeerAddresses(
@@ -125,27 +134,16 @@ public class Shard extends RaftActor {
         return map;
     }
 
-
-
-
     public static Props props(final ShardIdentifier name,
         final Map<ShardIdentifier, String> peerAddresses,
-        final InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
+        DatastoreContext datastoreContext) {
         Preconditions.checkNotNull(name, "name should not be null");
-        Preconditions
-            .checkNotNull(peerAddresses, "peerAddresses should not be null");
-
-        return Props.create(new Creator<Shard>() {
-
-            @Override
-            public Shard create() throws Exception {
-                return new Shard(name, peerAddresses, dataStoreProperties);
-            }
+        Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
+        Preconditions.checkNotNull(datastoreContext, "shardContext should not be null");
 
-        });
+        return Props.create(new ShardCreator(name, peerAddresses, datastoreContext));
     }
 
-
     @Override public void onReceiveCommand(Object message) {
         LOG.debug("Received message {} from {}", message.getClass().toString(),
             getSender());
@@ -188,9 +186,8 @@ public class Shard extends RaftActor {
             shardMBean.incrementReadOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(),
+                        schemaContext,datastoreContext, name.toString()), transactionId.toString());
 
         } else if (createTransaction.getTransactionType()
             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
@@ -198,9 +195,8 @@ public class Shard extends RaftActor {
             shardMBean.incrementReadWriteTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadWriteTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newReadWriteTransaction(), getSelf(),
+                        schemaContext, datastoreContext,name.toString()), transactionId.toString());
 
 
         } else if (createTransaction.getTransactionType()
@@ -209,9 +205,8 @@ public class Shard extends RaftActor {
             shardMBean.incrementWriteOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newWriteOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(),
+                        schemaContext, datastoreContext, name.toString()), transactionId.toString());
         } else {
             throw new IllegalArgumentException(
                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
@@ -269,12 +264,14 @@ public class Shard extends RaftActor {
         final ActorRef self = getSelf();
 
         Futures.addCallback(future, new FutureCallback<Void>() {
+            @Override
             public void onSuccess(Void v) {
                sender.tell(new CommitTransactionReply().toSerializable(),self);
                shardMBean.incrementCommittedTransactionCount();
                shardMBean.setLastCommittedTransactionTime(new Date());
             }
 
+            @Override
             public void onFailure(Throwable t) {
                 LOG.error(t, "An exception happened during commit");
                 shardMBean.incrementFailedTransactionsCount();
@@ -327,12 +324,10 @@ public class Shard extends RaftActor {
         dataChangeListeners.add(dataChangeListenerPath);
 
         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
-            listener =
-            new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
+            listener = new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
 
-        org.opendaylight.yangtools.concepts.ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
-            registration =
-            store.registerChangeListener(registerChangeListener.getPath(),
+        ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
+            registration = store.registerChangeListener(registerChangeListener.getPath(),
                 listener, registerChangeListener.getScope());
         ActorRef listenerRegistration =
             getContext().actorOf(
@@ -349,12 +344,9 @@ public class Shard extends RaftActor {
 
     private void createTransactionChain() {
         DOMStoreTransactionChain chain = store.createTransactionChain();
-        ActorRef transactionChain =
-            getContext().actorOf(
-                ShardTransactionChain.props(chain, schemaContext));
-        getSender()
-            .tell(new CreateTransactionChainReply(transactionChain.path())
-                    .toSerializable(),
+        ActorRef transactionChain = getContext().actorOf(
+                ShardTransactionChain.props(chain, schemaContext, datastoreContext,name.toString() ));
+        getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
                 getSelf());
     }
 
@@ -426,4 +418,25 @@ public class Shard extends RaftActor {
             return HEART_BEAT_INTERVAL;
         }
     }
+
+    private static class ShardCreator implements Creator<Shard> {
+
+        private static final long serialVersionUID = 1L;
+
+        final ShardIdentifier name;
+        final Map<ShardIdentifier, String> peerAddresses;
+        final DatastoreContext datastoreContext;
+
+        ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
+                DatastoreContext datastoreContext) {
+            this.name = name;
+            this.peerAddresses = peerAddresses;
+            this.datastoreContext = datastoreContext;
+        }
+
+        @Override
+        public Shard create() throws Exception {
+            return new Shard(name, peerAddresses, datastoreContext);
+        }
+    }
 }