Bug 1804: the condition was not working correctly to report leader
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
index 7d570046d406feec976620f9215398475711a756..f3f8b8b193c78a5511c02ec6ede526f632c3f172 100644 (file)
@@ -26,13 +26,15 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import org.opendaylight.controller.cluster.common.actor.CommonConfig;
+import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
 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.CloseTransactionChain;
 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.EnableNotification;
@@ -51,6 +53,7 @@ import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
 import org.opendaylight.controller.cluster.raft.RaftActor;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
+import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
@@ -59,6 +62,7 @@ import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessa
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
+import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
@@ -107,11 +111,12 @@ public class Shard extends RaftActor {
 
     private final DatastoreContext datastoreContext;
 
-
     private SchemaContext schemaContext;
 
     private ActorRef createSnapshotTransaction;
 
+    private final Map<String, DOMStoreTransactionChain> transactionChains = new HashMap<>();
+
     private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
             DatastoreContext datastoreContext, SchemaContext schemaContext) {
         super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
@@ -138,6 +143,9 @@ public class Shard extends RaftActor {
         shardMBean.setDataStoreExecutor(store.getDomStoreExecutor());
         shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager());
 
+        if (isMetricsCaptureEnabled()) {
+            getContext().become(new MeteringBehavior(this));
+        }
     }
 
     private static Map<String, String> mapPeerAddresses(
@@ -178,22 +186,20 @@ public class Shard extends RaftActor {
         LOG.debug("onReceiveCommand: Received message {} from {}", message.getClass().toString(),
             getSender());
 
-        if (message.getClass()
-            .equals(CreateTransactionChain.SERIALIZABLE_CLASS)) {
-            if (isLeader()) {
-                createTransactionChain();
-            } else if (getLeader() != null) {
-                getLeader().forward(message, getContext());
-            }
-        } else if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
+        if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
             // This must be for install snapshot. Don't want to open this up and trigger
             // deSerialization
-            self().tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)), self());
+            self()
+                .tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)),
+                    self());
 
+            createSnapshotTransaction = null;
             // Send a PoisonPill instead of sending close transaction because we do not really need
             // a response
             getSender().tell(PoisonPill.getInstance(), self());
 
+        } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){
+            closeTransactionChain(CloseTransactionChain.fromSerializable(message));
         } else if (message instanceof RegisterChangeListener) {
             registerChangeListener((RegisterChangeListener) message);
         } else if (message instanceof UpdateSchemaContext) {
@@ -206,6 +212,9 @@ public class Shard extends RaftActor {
                 createTransaction(CreateTransaction.fromSerializable(message));
             } else if (getLeader() != null) {
                 getLeader().forward(message, getContext());
+            } else {
+                getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(
+                    "Could not find leader so transaction cannot be created")), getSelf());
             }
         } else if (message instanceof PeerAddressResolved) {
             PeerAddressResolved resolved = (PeerAddressResolved) message;
@@ -216,9 +225,30 @@ public class Shard extends RaftActor {
         }
     }
 
+    private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
+        DOMStoreTransactionChain chain =
+            transactionChains.remove(closeTransactionChain.getTransactionChainId());
+
+        if(chain != null) {
+            chain.close();
+        }
+    }
+
     private ActorRef createTypedTransactionActor(
         int transactionType,
-        ShardTransactionIdentifier transactionId) {
+        ShardTransactionIdentifier transactionId,
+        String transactionChainId ) {
+
+        DOMStoreTransactionFactory factory = store;
+
+        if(!transactionChainId.isEmpty()) {
+            factory = transactionChains.get(transactionChainId);
+            if(factory == null){
+                DOMStoreTransactionChain transactionChain = store.createTransactionChain();
+                transactionChains.put(transactionChainId, transactionChain);
+                factory = transactionChain;
+            }
+        }
 
         if(this.schemaContext == null){
             throw new NullPointerException("schemaContext should not be null");
@@ -230,7 +260,7 @@ public class Shard extends RaftActor {
             shardMBean.incrementReadOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(),
+                ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
                         schemaContext,datastoreContext, shardMBean), transactionId.toString());
 
         } else if (transactionType
@@ -239,7 +269,7 @@ public class Shard extends RaftActor {
             shardMBean.incrementReadWriteTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction.props(store.newReadWriteTransaction(), getSelf(),
+                ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
 
 
@@ -249,7 +279,7 @@ public class Shard extends RaftActor {
             shardMBean.incrementWriteOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(),
+                ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
         } else {
             throw new IllegalArgumentException(
@@ -260,10 +290,10 @@ public class Shard extends RaftActor {
 
     private void createTransaction(CreateTransaction createTransaction) {
         createTransaction(createTransaction.getTransactionType(),
-            createTransaction.getTransactionId());
+            createTransaction.getTransactionId(), createTransaction.getTransactionChainId());
     }
 
-    private ActorRef createTransaction(int transactionType, String remoteTransactionId) {
+    private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) {
 
         ShardTransactionIdentifier transactionId =
             ShardTransactionIdentifier.builder()
@@ -271,7 +301,7 @@ public class Shard extends RaftActor {
                 .build();
         LOG.debug("Creating transaction : {} ", transactionId);
         ActorRef transactionActor =
-            createTypedTransactionActor(transactionType, transactionId);
+            createTypedTransactionActor(transactionType, transactionId, transactionChainId);
 
         getSender()
             .tell(new CreateTransactionReply(
@@ -301,6 +331,9 @@ public class Shard extends RaftActor {
                 modification);
             DOMStoreWriteTransaction transaction =
                 store.newWriteOnlyTransaction();
+
+            LOG.debug("Created new transaction {}", transaction.getIdentifier().toString());
+
             modification.apply(transaction);
             try {
                 syncCommitTransaction(transaction);
@@ -314,6 +347,12 @@ public class Shard extends RaftActor {
             return;
         }
 
+
+        if(sender == null){
+            LOG.error("Commit failed. Sender cannot be null");
+            return;
+        }
+
         final ListenableFuture<Void> future = cohort.commit();
         final ActorRef self = getSelf();
 
@@ -406,7 +445,12 @@ public class Shard extends RaftActor {
         ActorRef transactionChain = getContext().actorOf(
                 ShardTransactionChain.props(chain, schemaContext, datastoreContext, shardMBean));
         getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
-            getSelf());
+                getSelf());
+    }
+
+    private boolean isMetricsCaptureEnabled(){
+        CommonConfig config = new CommonConfig(getContext().system().settings().config());
+        return config.isMetricCaptureEnabled();
     }
 
     @Override protected void applyState(ActorRef clientActor, String identifier,
@@ -425,7 +469,7 @@ public class Shard extends RaftActor {
             }
 
         } else {
-            LOG.error("Unknown state received {}", data);
+            LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}", data, data.getClass().getClassLoader(), CompositeModificationPayload.class.getClassLoader());
         }
 
         // Update stats
@@ -448,7 +492,7 @@ public class Shard extends RaftActor {
             // so that this actor does not get block building the snapshot
             createSnapshotTransaction = createTransaction(
                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
-                "createSnapshot");
+                "createSnapshot", "");
 
             createSnapshotTransaction.tell(
                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
@@ -460,6 +504,8 @@ public class Shard extends RaftActor {
         // Since this will be done only on Recovery or when this actor is a Follower
         // we can safely commit everything in here. We not need to worry about event notifications
         // as they would have already been disabled on the follower
+
+        LOG.info("Applying snapshot");
         try {
             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
@@ -474,6 +520,8 @@ public class Shard extends RaftActor {
             syncCommitTransaction(transaction);
         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
             LOG.error(e, "An exception occurred when applying snapshot");
+        } finally {
+            LOG.info("Done applying snapshot");
         }
     }
 
@@ -483,12 +531,23 @@ public class Shard extends RaftActor {
                 .tell(new EnableNotification(isLeader()), getSelf());
         }
 
-        if (getLeaderId() != null) {
-            shardMBean.setLeader(getLeaderId());
-        }
 
         shardMBean.setRaftState(getRaftState().name());
         shardMBean.setCurrentTerm(getCurrentTerm());
+
+        // If this actor is no longer the leader close all the transaction chains
+        if(!isLeader()){
+            for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
+                LOG.debug("onStateChanged: Closing transaction chain {} because shard {} is no longer the leader", entry.getKey(), getId());
+                entry.getValue().close();
+            }
+
+            transactionChains.clear();
+        }
+    }
+
+    @Override protected void onLeaderChanged(String oldLeader, String newLeader) {
+        shardMBean.setLeader(newLeader);
     }
 
     @Override public String persistenceId() {