Merge "Serialization/Deserialization and a host of other fixes"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
index 5b4f7ef8989711dffdf4cdd8fbe7d483165f23a7..d6ad553cf3a237002645811dad1b299c3f2fd309 100644 (file)
@@ -15,24 +15,33 @@ import akka.event.Logging;
 import akka.event.LoggingAdapter;
 import akka.japi.Creator;
 import akka.persistence.Persistent;
+import akka.persistence.RecoveryCompleted;
 import akka.persistence.UntypedProcessor;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+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;
+import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChain;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
+import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.NonPersistent;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
+import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
+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.yang.data.api.InstanceIdentifier;
+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 java.util.HashMap;
 import java.util.Map;
@@ -54,14 +63,32 @@ public class Shard extends UntypedProcessor {
 
     private final InMemoryDOMDataStore store;
 
-    private final Map<Modification, DOMStoreThreePhaseCommitCohort>
+    private final Map<Object, DOMStoreThreePhaseCommitCohort>
         modificationToCohort = new HashMap<>();
 
-    private final LoggingAdapter log =
+    private final LoggingAdapter LOG =
         Logging.getLogger(getContext().system(), this);
 
+    // By default persistent will be true and can be turned off using the system
+    // property persistent
+    private final boolean persistent;
+
+    private SchemaContext schemaContext;
+
+    private final ShardStats shardMBean;
+
     private Shard(String name) {
+
+        String setting = System.getProperty("shard.persistent");
+
+        this.persistent = !"false".equals(setting);
+
+        LOG.info("Creating shard : {} persistent : {}", name, persistent);
+
         store = new InMemoryDOMDataStore(name, storeExecutor);
+
+        shardMBean = ShardMBeanFactory.getShardStatsMBean(name);
+
     }
 
     public static Props props(final String name) {
@@ -75,31 +102,59 @@ public class Shard extends UntypedProcessor {
         });
     }
 
+
     @Override
     public void onReceive(Object message) throws Exception {
-        if (message instanceof CreateTransactionChain) {
+        LOG.debug("Received message " + message.getClass().toString());
+
+        if(!recoveryFinished()){
+            // FIXME : Properly handle recovery
+            return;
+        }
+
+        if (message.getClass().equals(CreateTransactionChain.SERIALIZABLE_CLASS)) {
             createTransactionChain();
-        } else if (message instanceof RegisterChangeListener) {
-            registerChangeListener((RegisterChangeListener) message);
+        } else if (message.getClass().equals(RegisterChangeListener.SERIALIZABLE_CLASS)) {
+            registerChangeListener(RegisterChangeListener.fromSerializable(getContext().system(), message));
         } else if (message instanceof UpdateSchemaContext) {
             updateSchemaContext((UpdateSchemaContext) message);
         } else if (message instanceof ForwardedCommitTransaction) {
             handleForwardedCommit((ForwardedCommitTransaction) message);
         } else if (message instanceof Persistent) {
-            commit((Persistent) message);
+            commit(((Persistent)message).payload());
+        } else if (message.getClass().equals(CreateTransaction.SERIALIZABLE_CLASS)) {
+            createTransaction(CreateTransaction.fromSerializable(message));
+        } else if(message instanceof NonPersistent){
+            commit(((NonPersistent)message).payload());
+        }else if (message instanceof RecoveryCompleted) {
+            //FIXME: PROPERLY HANDLE RECOVERY COMPLETED
+
+        }else {
+          throw new Exception("Not recognized message found message=" + message);
         }
     }
 
-    private void commit(Persistent message) {
-        Modification modification = (Modification) message.payload();
+    private void createTransaction(CreateTransaction createTransaction) {
+        DOMStoreReadWriteTransaction transaction =
+            store.newReadWriteTransaction();
+        ActorRef transactionActor = getContext().actorOf(
+            ShardTransaction.props(transaction, getSelf(), schemaContext), "shard-" + createTransaction.getTransactionId());
+        getSender()
+            .tell(new CreateTransactionReply(transactionActor.path().toString(), createTransaction.getTransactionId()).toSerializable(),
+                getSelf());
+    }
+
+    private void commit(Object serialized) {
+        Modification modification = MutableCompositeModification.fromSerializable(serialized, schemaContext);
         DOMStoreThreePhaseCommitCohort cohort =
-            modificationToCohort.remove(modification);
+            modificationToCohort.remove(serialized);
         if (cohort == null) {
-            log.error(
+            LOG.error(
                 "Could not find cohort for modification : " + modification);
             return;
         }
         final ListenableFuture<Void> future = cohort.commit();
+        shardMBean.incrementCommittedTransactionCount();
         final ActorRef sender = getSender();
         final ActorRef self = getSelf();
         future.addListener(new Runnable() {
@@ -107,52 +162,67 @@ public class Shard extends UntypedProcessor {
             public void run() {
                 try {
                     future.get();
-                    sender.tell(new CommitTransactionReply(), self);
+                    sender.tell(new CommitTransactionReply().toSerializable(), self);
                 } catch (InterruptedException | ExecutionException e) {
-                    log.error(e, "An exception happened when committing");
+                    // FIXME : Handle this properly
+                    LOG.error(e, "An exception happened when committing");
                 }
             }
         }, getContext().dispatcher());
     }
 
     private void handleForwardedCommit(ForwardedCommitTransaction message) {
-        log.info("received forwarded transaction");
+        Object serializedModification = message.getModification().toSerializable();
+
         modificationToCohort
-            .put(message.getModification(), message.getCohort());
-        getSelf().forward(Persistent.create(message.getModification()),
-            getContext());
+            .put(serializedModification , message.getCohort());
+        if(persistent) {
+            getSelf().forward(Persistent.create(serializedModification),
+                getContext());
+        } else {
+            getSelf().forward(NonPersistent.create(serializedModification),
+                getContext());
+        }
     }
 
     private void updateSchemaContext(UpdateSchemaContext message) {
+        this.schemaContext = message.getSchemaContext();
         store.onGlobalContextUpdated(message.getSchemaContext());
     }
 
     private void registerChangeListener(
         RegisterChangeListener registerChangeListener) {
 
-        ActorSelection listenerRegistrationActor = getContext()
+        LOG.debug("registerDataChangeListener for " + registerChangeListener.getPath());
+
+
+        ActorSelection dataChangeListenerPath = getContext()
             .system().actorSelection(registerChangeListener.getDataChangeListenerPath());
 
-        AsyncDataChangeListener<InstanceIdentifier, NormalizedNode<?, ?>>
-            listener = new ListenerProxy(listenerRegistrationActor);
+        AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
+            listener = new DataChangeListenerProxy(schemaContext,dataChangeListenerPath);
 
-        org.opendaylight.yangtools.concepts.ListenerRegistration<AsyncDataChangeListener<InstanceIdentifier, NormalizedNode<?, ?>>>
+        org.opendaylight.yangtools.concepts.ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
             registration =
             store.registerChangeListener(registerChangeListener.getPath(),
                 listener, registerChangeListener.getScope());
         ActorRef listenerRegistration =
-            getContext().actorOf(ListenerRegistration.props(registration));
+            getContext().actorOf(
+                DataChangeListenerRegistration.props(registration));
+
+        LOG.debug("registerDataChangeListener sending reply, listenerRegistrationPath = " + listenerRegistration.path().toString());
+
         getSender()
-            .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
+            .tell(new RegisterChangeListenerReply(listenerRegistration.path()).toSerializable(),
                 getSelf());
     }
 
     private void createTransactionChain() {
         DOMStoreTransactionChain chain = store.createTransactionChain();
         ActorRef transactionChain =
-            getContext().actorOf(ShardTransactionChain.props(chain));
+            getContext().actorOf(ShardTransactionChain.props(chain, schemaContext));
         getSender()
-            .tell(new CreateTransactionChainReply(transactionChain.path()),
+            .tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
                 getSelf());
     }
 }