Include JMX Counters and resetTransactionCounters
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
index 308589e7f0b9be1ea41eb6679e2218cab07762d1..43a9faa3e44e5d6fe77db7cce74929f8888dcadb 100644 (file)
@@ -15,8 +15,15 @@ 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;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
@@ -42,9 +49,11 @@ import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFac
 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;
@@ -67,6 +76,7 @@ public class Shard extends RaftActor {
 
     public static final String DEFAULT_NAME = "default";
 
+    // The state of this Shard
     private final InMemoryDOMDataStore store;
 
     private final Map<Object, DOMStoreThreePhaseCommitCohort>
@@ -76,46 +86,63 @@ public class Shard extends RaftActor {
         Logging.getLogger(getContext().system(), this);
 
     // By default persistent will be true and can be turned off using the system
-    // property persistent
+    // property shard.persistent
     private final boolean persistent;
 
-    private final String name;
-
-    private volatile SchemaContext schemaContext;
+    /// The name of this shard
+    private final ShardIdentifier name;
 
     private final ShardStats shardMBean;
 
     private final List<ActorSelection> dataChangeListeners = new ArrayList<>();
 
-    private Shard(String name, Map<String, String> peerAddresses) {
-        super(name, peerAddresses, Optional.of(configParams));
+    private final DatastoreContext datastoreContext;
+
+
+    private SchemaContext schemaContext;
+
+    private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
+            DatastoreContext datastoreContext) {
+        super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
 
         this.name = name;
+        this.datastoreContext = datastoreContext;
 
         String setting = System.getProperty("shard.persistent");
 
         this.persistent = !"false".equals(setting);
 
-        LOG.info("Creating shard : {} persistent : {}", name, persistent);
+        LOG.info("Shard created : {} persistent : {}", name, persistent);
+
+        store = InMemoryDOMDataStoreFactory.create(name.toString(), null,
+                datastoreContext.getDataStoreProperties());
 
-        store = InMemoryDOMDataStoreFactory.create(name, null);
+        shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString());
 
-        shardMBean = ShardMBeanFactory.getShardStatsMBean(name);
 
     }
 
-    public static Props props(final String name,
-        final Map<String, String> peerAddresses) {
-        return Props.create(new Creator<Shard>() {
+    private static Map<String, String> mapPeerAddresses(
+        Map<ShardIdentifier, String> peerAddresses) {
+        Map<String, String> map = new HashMap<>();
 
-            @Override
-            public Shard create() throws Exception {
-                return new Shard(name, peerAddresses);
-            }
+        for (Map.Entry<ShardIdentifier, String> entry : peerAddresses
+            .entrySet()) {
+            map.put(entry.getKey().toString(), entry.getValue());
+        }
 
-        });
+        return map;
     }
 
+    public static Props props(final ShardIdentifier name,
+        final Map<ShardIdentifier, String> peerAddresses,
+        DatastoreContext datastoreContext) {
+        Preconditions.checkNotNull(name, "name should not be null");
+        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(),
@@ -143,56 +170,64 @@ public class Shard extends RaftActor {
             }
         } else if (message instanceof PeerAddressResolved) {
             PeerAddressResolved resolved = (PeerAddressResolved) message;
-            setPeerAddress(resolved.getPeerId(), resolved.getPeerAddress());
+            setPeerAddress(resolved.getPeerId().toString(),
+                resolved.getPeerAddress());
         } else {
             super.onReceiveCommand(message);
         }
     }
 
     private ActorRef createTypedTransactionActor(
-        CreateTransaction createTransaction, String transactionId) {
+        CreateTransaction createTransaction,
+        ShardTransactionIdentifier transactionId) {
         if (createTransaction.getTransactionType()
             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
+
             shardMBean.incrementReadOnlyTransactionCount();
+
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId);
+                ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(),
+                        schemaContext,datastoreContext, name.toString()), transactionId.toString());
 
         } else if (createTransaction.getTransactionType()
             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
+
             shardMBean.incrementReadWriteTransactionCount();
+
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadWriteTransaction(), getSelf(),
-                        schemaContext), transactionId);
+                ShardTransaction.props(store.newReadWriteTransaction(), getSelf(),
+                        schemaContext, datastoreContext,name.toString()), transactionId.toString());
 
 
         } else if (createTransaction.getTransactionType()
             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
+
             shardMBean.incrementWriteOnlyTransactionCount();
+
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newWriteOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId);
+                ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(),
+                        schemaContext, datastoreContext, name.toString()), transactionId.toString());
         } else {
             throw new IllegalArgumentException(
-                "CreateTransaction message has unidentified transaction type="
+                "Shard="+name + ":CreateTransaction message has unidentified transaction type="
                     + createTransaction.getTransactionType());
         }
     }
 
     private void createTransaction(CreateTransaction createTransaction) {
 
-        String transactionId = "shard-" + createTransaction.getTransactionId();
-        LOG.info("Creating transaction : {} ", transactionId);
+        ShardTransactionIdentifier transactionId =
+            ShardTransactionIdentifier.builder()
+                .remoteTransactionId(createTransaction.getTransactionId())
+                .build();
+        LOG.debug("Creating transaction : {} ", transactionId);
         ActorRef transactionActor =
             createTypedTransactionActor(createTransaction, transactionId);
 
         getSender()
             .tell(new CreateTransactionReply(
-                Serialization.serializedActorPath(transactionActor),
-                createTransaction.getTransactionId()).toSerializable(),
+                    Serialization.serializedActorPath(transactionActor),
+                    createTransaction.getTransactionId()).toSerializable(),
                 getSelf());
     }
 
@@ -202,9 +237,9 @@ public class Shard extends RaftActor {
         DOMStoreThreePhaseCommitCohort cohort =
             modificationToCohort.remove(serialized);
         if (cohort == null) {
-            LOG.error(
-                "Could not find cohort for modification : {}", modification);
-            LOG.info("Writing modification using a new transaction");
+            LOG.debug(
+                "Could not find cohort for modification : {}. Writing modification using a new transaction",
+                modification);
             DOMStoreReadWriteTransaction transaction =
                 store.newReadWriteTransaction();
             modification.apply(transaction);
@@ -227,24 +262,23 @@ public class Shard extends RaftActor {
 
         final ListenableFuture<Void> future = cohort.commit();
         final ActorRef self = getSelf();
-        future.addListener(new Runnable() {
+
+        Futures.addCallback(future, new FutureCallback<Void>() {
             @Override
-            public void run() {
-                try {
-                    future.get();
-                        sender
-                            .tell(new CommitTransactionReply().toSerializable(),
-                                self);
-                        shardMBean.incrementCommittedTransactionCount();
-                        shardMBean.setLastCommittedTransactionTime(new Date());
-
-                } catch (InterruptedException | ExecutionException e) {
-                    shardMBean.incrementFailedTransactionsCount();
-                    // FIXME : Handle this properly
-                    LOG.error(e, "An exception happened when committing");
-                }
+            public void onSuccess(Void v) {
+               sender.tell(new CommitTransactionReply().toSerializable(),self);
+               shardMBean.incrementCommittedTransactionCount();
+               shardMBean.setLastCommittedTransactionTime(new Date());
             }
-        }, getContext().dispatcher());
+
+            @Override
+            public void onFailure(Throwable t) {
+                LOG.error(t, "An exception happened during commit");
+                shardMBean.incrementFailedTransactionsCount();
+                sender.tell(new akka.actor.Status.Failure(t), self);
+            }
+        });
+
     }
 
     private void handleForwardedCommit(ForwardedCommitTransaction message) {
@@ -270,7 +304,7 @@ public class Shard extends RaftActor {
     private void registerChangeListener(
         RegisterChangeListener registerChangeListener) {
 
-        LOG.debug("registerDataChangeListener for " + registerChangeListener
+        LOG.debug("registerDataChangeListener for {}", registerChangeListener
             .getPath());
 
 
@@ -290,20 +324,18 @@ 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(
                 DataChangeListenerRegistration.props(registration));
 
         LOG.debug(
-            "registerDataChangeListener sending reply, listenerRegistrationPath = "
-                + listenerRegistration.path().toString());
+            "registerDataChangeListener sending reply, listenerRegistrationPath = {} "
+            , listenerRegistration.path().toString());
 
         getSender()
             .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
@@ -312,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());
     }
 
@@ -331,7 +360,9 @@ public class Shard extends RaftActor {
             if (modification != null) {
                 commit(clientActor, modification);
             } else {
-                LOG.error("modification is null - this is very unexpected");
+                LOG.error(
+                    "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
+                    identifier, clientActor.path().toString());
             }
 
 
@@ -339,9 +370,10 @@ public class Shard extends RaftActor {
             LOG.error("Unknown state received {}", data);
         }
 
+        // Update stats
         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
 
-        if(lastLogEntry != null){
+        if (lastLogEntry != null) {
             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
         }
@@ -374,7 +406,7 @@ public class Shard extends RaftActor {
     }
 
     @Override public String persistenceId() {
-        return this.name;
+        return this.name.toString();
     }
 
 
@@ -386,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);
+        }
+    }
 }