Merge "BUG-1690: catch wildcard InstanceIdentifiers"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
index 75f540ade088e6bb45b10e9b92bc4af8789b0218..7d570046d406feec976620f9215398475711a756 100644 (file)
@@ -10,16 +10,22 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.PoisonPill;
 import akka.actor.Props;
 import akka.event.Logging;
 import akka.event.LoggingAdapter;
 import akka.japi.Creator;
+import akka.persistence.RecoveryFailure;
 import akka.serialization.Serialization;
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.FutureCallback;
 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.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
@@ -32,29 +38,35 @@ import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionR
 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
+import org.opendaylight.controller.cluster.datastore.messages.ReadData;
+import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
 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.cluster.datastore.node.NormalizedNodeToNodeCodec;
 import org.opendaylight.controller.cluster.raft.ConfigParams;
 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.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;
-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.protobuff.messages.common.NormalizedNodeMessages;
+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.DOMStoreWriteTransaction;
+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;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -89,17 +101,24 @@ 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 ActorRef createSnapshotTransaction;
+
     private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
-            InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
+            DatastoreContext datastoreContext, SchemaContext schemaContext) {
         super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
 
         this.name = name;
+        this.datastoreContext = datastoreContext;
+        this.schemaContext = schemaContext;
 
         String setting = System.getProperty("shard.persistent");
 
@@ -107,9 +126,17 @@ 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());
+
+        if(schemaContext != null) {
+            store.onGlobalContextUpdated(schemaContext);
+        }
 
-        shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString());
+        shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(),
+                datastoreContext.getDataStoreMXBeanType());
+        shardMBean.setDataStoreExecutor(store.getDomStoreExecutor());
+        shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager());
 
     }
 
@@ -125,29 +152,30 @@ public class Shard extends RaftActor {
         return map;
     }
 
-
-
-
     public static Props props(final ShardIdentifier name,
         final Map<ShardIdentifier, String> peerAddresses,
-        final InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
+        DatastoreContext datastoreContext, SchemaContext schemaContext) {
         Preconditions.checkNotNull(name, "name should not be null");
-        Preconditions
-            .checkNotNull(peerAddresses, "peerAddresses should not be null");
+        Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
+        Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
+        Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
 
-        return Props.create(new Creator<Shard>() {
+        return Props.create(new ShardCreator(name, peerAddresses, datastoreContext, schemaContext));
+    }
 
-            @Override
-            public Shard create() throws Exception {
-                return new Shard(name, peerAddresses, dataStoreProperties);
-            }
+    @Override public void onReceiveRecover(Object message) {
+        LOG.debug("onReceiveRecover: Received message {} from {}", message.getClass().toString(),
+            getSender());
 
-        });
+        if (message instanceof RecoveryFailure){
+            LOG.error(((RecoveryFailure) message).cause(), "Recovery failed because of this cause");
+        } else {
+            super.onReceiveRecover(message);
+        }
     }
 
-
     @Override public void onReceiveCommand(Object message) {
-        LOG.debug("Received message {} from {}", message.getClass().toString(),
+        LOG.debug("onReceiveCommand: Received message {} from {}", message.getClass().toString(),
             getSender());
 
         if (message.getClass()
@@ -157,6 +185,15 @@ public class Shard extends RaftActor {
             } else if (getLeader() != null) {
                 getLeader().forward(message, getContext());
             }
+        } else 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());
+
+            // Send a PoisonPill instead of sending close transaction because we do not really need
+            // a response
+            getSender().tell(PoisonPill.getInstance(), self());
+
         } else if (message instanceof RegisterChangeListener) {
             registerChangeListener((RegisterChangeListener) message);
         } else if (message instanceof UpdateSchemaContext) {
@@ -180,62 +217,79 @@ public class Shard extends RaftActor {
     }
 
     private ActorRef createTypedTransactionActor(
-        CreateTransaction createTransaction,
+        int transactionType,
         ShardTransactionIdentifier transactionId) {
-        if (createTransaction.getTransactionType()
+
+        if(this.schemaContext == null){
+            throw new NullPointerException("schemaContext should not be null");
+        }
+
+        if (transactionType
             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
 
             shardMBean.incrementReadOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newReadOnlyTransaction(), getSelf(),
+                        schemaContext,datastoreContext, shardMBean), transactionId.toString());
 
-        } else if (createTransaction.getTransactionType()
+        } else if (transactionType
             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
 
             shardMBean.incrementReadWriteTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newReadWriteTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newReadWriteTransaction(), getSelf(),
+                        schemaContext, datastoreContext, shardMBean), transactionId.toString());
 
 
-        } else if (createTransaction.getTransactionType()
+        } else if (transactionType
             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
 
             shardMBean.incrementWriteOnlyTransactionCount();
 
             return getContext().actorOf(
-                ShardTransaction
-                    .props(store.newWriteOnlyTransaction(), getSelf(),
-                        schemaContext), transactionId.toString());
+                ShardTransaction.props(store.newWriteOnlyTransaction(), getSelf(),
+                        schemaContext, datastoreContext, shardMBean), transactionId.toString());
         } else {
             throw new IllegalArgumentException(
                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
-                    + createTransaction.getTransactionType());
+                    + transactionType);
         }
     }
 
     private void createTransaction(CreateTransaction createTransaction) {
+        createTransaction(createTransaction.getTransactionType(),
+            createTransaction.getTransactionId());
+    }
+
+    private ActorRef createTransaction(int transactionType, String remoteTransactionId) {
 
         ShardTransactionIdentifier transactionId =
             ShardTransactionIdentifier.builder()
-                .remoteTransactionId(createTransaction.getTransactionId())
+                .remoteTransactionId(remoteTransactionId)
                 .build();
         LOG.debug("Creating transaction : {} ", transactionId);
         ActorRef transactionActor =
-            createTypedTransactionActor(createTransaction, transactionId);
+            createTypedTransactionActor(transactionType, transactionId);
 
         getSender()
             .tell(new CreateTransactionReply(
                     Serialization.serializedActorPath(transactionActor),
-                    createTransaction.getTransactionId()).toSerializable(),
+                    remoteTransactionId).toSerializable(),
                 getSelf());
+
+        return transactionActor;
+    }
+
+    private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
+        throws ExecutionException, InterruptedException {
+        DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
+        commitCohort.preCommit().get();
+        commitCohort.commit().get();
     }
 
+
     private void commit(final ActorRef sender, Object serialized) {
         Modification modification = MutableCompositeModification
             .fromSerializable(serialized, schemaContext);
@@ -245,16 +299,11 @@ public class Shard extends RaftActor {
             LOG.debug(
                 "Could not find cohort for modification : {}. Writing modification using a new transaction",
                 modification);
-            DOMStoreReadWriteTransaction transaction =
-                store.newReadWriteTransaction();
+            DOMStoreWriteTransaction transaction =
+                store.newWriteOnlyTransaction();
             modification.apply(transaction);
-            DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
-            ListenableFuture<Void> future =
-                commitCohort.preCommit();
             try {
-                future.get();
-                future = commitCohort.commit();
-                future.get();
+                syncCommitTransaction(transaction);
             } catch (InterruptedException | ExecutionException e) {
                 shardMBean.incrementFailedTransactionsCount();
                 LOG.error("Failed to commit", e);
@@ -269,12 +318,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());
+                sender.tell(new CommitTransactionReply().toSerializable(), self);
+                shardMBean.incrementCommittedTransactionCount();
+                shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
             }
 
+            @Override
             public void onFailure(Throwable t) {
                 LOG.error(t, "An exception happened during commit");
                 shardMBean.incrementFailedTransactionsCount();
@@ -301,9 +352,14 @@ public class Shard extends RaftActor {
 
     private void updateSchemaContext(UpdateSchemaContext message) {
         this.schemaContext = message.getSchemaContext();
+        updateSchemaContext(message.getSchemaContext());
         store.onGlobalContextUpdated(message.getSchemaContext());
     }
 
+    @VisibleForTesting void updateSchemaContext(SchemaContext schemaContext) {
+        store.onGlobalContextUpdated(schemaContext);
+    }
+
     private void registerChangeListener(
         RegisterChangeListener registerChangeListener) {
 
@@ -327,12 +383,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,13 +403,10 @@ 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(),
-                getSelf());
+        ActorRef transactionChain = getContext().actorOf(
+                ShardTransactionChain.props(chain, schemaContext, datastoreContext, shardMBean));
+        getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
+            getSelf());
     }
 
     @Override protected void applyState(ActorRef clientActor, String identifier,
@@ -373,7 +424,6 @@ public class Shard extends RaftActor {
                     identifier, clientActor.path().toString());
             }
 
-
         } else {
             LOG.error("Unknown state received {}", data);
         }
@@ -391,12 +441,40 @@ public class Shard extends RaftActor {
 
     }
 
-    @Override protected Object createSnapshot() {
-        throw new UnsupportedOperationException("createSnapshot");
+    @Override protected void createSnapshot() {
+        if (createSnapshotTransaction == null) {
+
+            // Create a transaction. We are really going to treat the transaction as a worker
+            // so that this actor does not get block building the snapshot
+            createSnapshotTransaction = createTransaction(
+                TransactionProxy.TransactionType.READ_ONLY.ordinal(),
+                "createSnapshot");
+
+            createSnapshotTransaction.tell(
+                new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
+
+        }
     }
 
-    @Override protected void applySnapshot(Object snapshot) {
-        throw new UnsupportedOperationException("applySnapshot");
+    @VisibleForTesting @Override protected void applySnapshot(ByteString snapshot) {
+        // 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
+        try {
+            DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
+            NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
+            NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
+                .decode(YangInstanceIdentifier.builder().build(), serializedNode);
+
+            // delete everything first
+            transaction.delete(YangInstanceIdentifier.builder().build());
+
+            // Add everything from the remote node back
+            transaction.write(YangInstanceIdentifier.builder().build(), node);
+            syncCommitTransaction(transaction);
+        } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
+            LOG.error(e, "An exception occurred when applying snapshot");
+        }
     }
 
     @Override protected void onStateChanged() {
@@ -426,4 +504,50 @@ 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;
+        final SchemaContext schemaContext;
+
+        ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
+                DatastoreContext datastoreContext, SchemaContext schemaContext) {
+            this.name = name;
+            this.peerAddresses = peerAddresses;
+            this.datastoreContext = datastoreContext;
+            this.schemaContext = schemaContext;
+        }
+
+        @Override
+        public Shard create() throws Exception {
+            return new Shard(name, peerAddresses, datastoreContext, schemaContext);
+        }
+    }
+
+    @VisibleForTesting NormalizedNode readStore() throws ExecutionException, InterruptedException {
+        DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
+
+        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
+            transaction.read(YangInstanceIdentifier.builder().build());
+
+        NormalizedNode<?, ?> node = future.get().get();
+
+        transaction.close();
+
+        return node;
+    }
+
+    @VisibleForTesting void writeToStore(YangInstanceIdentifier id, NormalizedNode node)
+        throws ExecutionException, InterruptedException {
+        DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
+
+        transaction.write(id, node);
+
+        syncCommitTransaction(transaction);
+    }
+
 }