Merge "BUG-1792: share QNames in both cases"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.datastore;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.PoisonPill;
14 import akka.actor.Props;
15 import akka.event.Logging;
16 import akka.event.LoggingAdapter;
17 import akka.japi.Creator;
18 import akka.persistence.RecoveryFailure;
19 import akka.serialization.Serialization;
20 import com.google.common.annotations.VisibleForTesting;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Preconditions;
23 import com.google.common.util.concurrent.CheckedFuture;
24 import com.google.common.util.concurrent.FutureCallback;
25 import com.google.common.util.concurrent.Futures;
26 import com.google.common.util.concurrent.ListenableFuture;
27 import com.google.protobuf.ByteString;
28 import com.google.protobuf.InvalidProtocolBufferException;
29 import org.opendaylight.controller.cluster.common.actor.CommonConfig;
30 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
31 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
32 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
33 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
34 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
35 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
36 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
37 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
38 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
39 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
40 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
41 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
42 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
43 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
44 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
45 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
46 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
47 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
48 import org.opendaylight.controller.cluster.datastore.modification.Modification;
49 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
50 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
51 import org.opendaylight.controller.cluster.raft.ConfigParams;
52 import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
53 import org.opendaylight.controller.cluster.raft.RaftActor;
54 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
55 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
56 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
57 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
58 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
59 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
60 import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
61 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
62 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
63 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
64 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory;
65 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
66 import org.opendaylight.yangtools.concepts.ListenerRegistration;
67 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
68 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
69 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
70 import scala.concurrent.duration.FiniteDuration;
71
72 import java.util.ArrayList;
73 import java.util.HashMap;
74 import java.util.List;
75 import java.util.Map;
76 import java.util.concurrent.ExecutionException;
77 import java.util.concurrent.TimeUnit;
78
79 /**
80  * A Shard represents a portion of the logical data tree <br/>
81  * <p>
82  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
83  * </p>
84  */
85 public class Shard extends RaftActor {
86
87     private static final ConfigParams configParams = new ShardConfigParams();
88
89     public static final String DEFAULT_NAME = "default";
90
91     // The state of this Shard
92     private final InMemoryDOMDataStore store;
93
94     private final Map<Object, DOMStoreThreePhaseCommitCohort>
95         modificationToCohort = new HashMap<>();
96
97     private final LoggingAdapter LOG =
98         Logging.getLogger(getContext().system(), this);
99
100     // By default persistent will be true and can be turned off using the system
101     // property shard.persistent
102     private final boolean persistent;
103
104     /// The name of this shard
105     private final ShardIdentifier name;
106
107     private final ShardStats shardMBean;
108
109     private final List<ActorSelection> dataChangeListeners = new ArrayList<>();
110
111     private final DatastoreContext datastoreContext;
112
113     private SchemaContext schemaContext;
114
115     private ActorRef createSnapshotTransaction;
116
117     private final Map<String, DOMStoreTransactionChain> transactionChains = new HashMap<>();
118
119     private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
120             DatastoreContext datastoreContext, SchemaContext schemaContext) {
121         super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
122
123         this.name = name;
124         this.datastoreContext = datastoreContext;
125         this.schemaContext = schemaContext;
126
127         String setting = System.getProperty("shard.persistent");
128
129         this.persistent = !"false".equals(setting);
130
131         LOG.info("Shard created : {} persistent : {}", name, persistent);
132
133         store = InMemoryDOMDataStoreFactory.create(name.toString(), null,
134                 datastoreContext.getDataStoreProperties());
135
136         if(schemaContext != null) {
137             store.onGlobalContextUpdated(schemaContext);
138         }
139
140         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(),
141                 datastoreContext.getDataStoreMXBeanType());
142         shardMBean.setDataStoreExecutor(store.getDomStoreExecutor());
143         shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager());
144
145         if (isMetricsCaptureEnabled()) {
146             getContext().become(new MeteringBehavior(this));
147         }
148     }
149
150     private static Map<String, String> mapPeerAddresses(
151         Map<ShardIdentifier, String> peerAddresses) {
152         Map<String, String> map = new HashMap<>();
153
154         for (Map.Entry<ShardIdentifier, String> entry : peerAddresses
155             .entrySet()) {
156             map.put(entry.getKey().toString(), entry.getValue());
157         }
158
159         return map;
160     }
161
162     public static Props props(final ShardIdentifier name,
163         final Map<ShardIdentifier, String> peerAddresses,
164         DatastoreContext datastoreContext, SchemaContext schemaContext) {
165         Preconditions.checkNotNull(name, "name should not be null");
166         Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
167         Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
168         Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
169
170         return Props.create(new ShardCreator(name, peerAddresses, datastoreContext, schemaContext));
171     }
172
173     @Override public void onReceiveRecover(Object message) {
174         LOG.debug("onReceiveRecover: Received message {} from {}", message.getClass().toString(),
175             getSender());
176
177         if (message instanceof RecoveryFailure){
178             LOG.error(((RecoveryFailure) message).cause(), "Recovery failed because of this cause");
179         } else {
180             super.onReceiveRecover(message);
181         }
182     }
183
184     @Override public void onReceiveCommand(Object message) {
185         LOG.debug("onReceiveCommand: Received message {} from {}", message.getClass().toString(),
186             getSender());
187
188         if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
189             // This must be for install snapshot. Don't want to open this up and trigger
190             // deSerialization
191             self()
192                 .tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)),
193                     self());
194
195             // Send a PoisonPill instead of sending close transaction because we do not really need
196             // a response
197             getSender().tell(PoisonPill.getInstance(), self());
198
199         } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){
200             closeTransactionChain(CloseTransactionChain.fromSerializable(message));
201         } else if (message instanceof RegisterChangeListener) {
202             registerChangeListener((RegisterChangeListener) message);
203         } else if (message instanceof UpdateSchemaContext) {
204             updateSchemaContext((UpdateSchemaContext) message);
205         } else if (message instanceof ForwardedCommitTransaction) {
206             handleForwardedCommit((ForwardedCommitTransaction) message);
207         } else if (message.getClass()
208             .equals(CreateTransaction.SERIALIZABLE_CLASS)) {
209             if (isLeader()) {
210                 createTransaction(CreateTransaction.fromSerializable(message));
211             } else if (getLeader() != null) {
212                 getLeader().forward(message, getContext());
213             } else {
214                 getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(
215                     "Could not find leader so transaction cannot be created")), getSelf());
216             }
217         } else if (message instanceof PeerAddressResolved) {
218             PeerAddressResolved resolved = (PeerAddressResolved) message;
219             setPeerAddress(resolved.getPeerId().toString(),
220                 resolved.getPeerAddress());
221         } else {
222             super.onReceiveCommand(message);
223         }
224     }
225
226     private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
227         DOMStoreTransactionChain chain =
228             transactionChains.remove(closeTransactionChain.getTransactionChainId());
229
230         if(chain != null) {
231             chain.close();
232         }
233     }
234
235     private ActorRef createTypedTransactionActor(
236         int transactionType,
237         ShardTransactionIdentifier transactionId,
238         String transactionChainId ) {
239
240         DOMStoreTransactionFactory factory = store;
241
242         if(!transactionChainId.isEmpty()) {
243             factory = transactionChains.get(transactionChainId);
244             if(factory == null){
245                 DOMStoreTransactionChain transactionChain = store.createTransactionChain();
246                 transactionChains.put(transactionChainId, transactionChain);
247                 factory = transactionChain;
248             }
249         }
250
251         if(this.schemaContext == null){
252             throw new NullPointerException("schemaContext should not be null");
253         }
254
255         if (transactionType
256             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
257
258             shardMBean.incrementReadOnlyTransactionCount();
259
260             return getContext().actorOf(
261                 ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
262                         schemaContext,datastoreContext, shardMBean), transactionId.toString());
263
264         } else if (transactionType
265             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
266
267             shardMBean.incrementReadWriteTransactionCount();
268
269             return getContext().actorOf(
270                 ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
271                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
272
273
274         } else if (transactionType
275             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
276
277             shardMBean.incrementWriteOnlyTransactionCount();
278
279             return getContext().actorOf(
280                 ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
281                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
282         } else {
283             throw new IllegalArgumentException(
284                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
285                     + transactionType);
286         }
287     }
288
289     private void createTransaction(CreateTransaction createTransaction) {
290         createTransaction(createTransaction.getTransactionType(),
291             createTransaction.getTransactionId(), createTransaction.getTransactionChainId());
292     }
293
294     private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) {
295
296         ShardTransactionIdentifier transactionId =
297             ShardTransactionIdentifier.builder()
298                 .remoteTransactionId(remoteTransactionId)
299                 .build();
300         LOG.debug("Creating transaction : {} ", transactionId);
301         ActorRef transactionActor =
302             createTypedTransactionActor(transactionType, transactionId, transactionChainId);
303
304         getSender()
305             .tell(new CreateTransactionReply(
306                     Serialization.serializedActorPath(transactionActor),
307                     remoteTransactionId).toSerializable(),
308                 getSelf());
309
310         return transactionActor;
311     }
312
313     private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
314         throws ExecutionException, InterruptedException {
315         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
316         commitCohort.preCommit().get();
317         commitCohort.commit().get();
318     }
319
320
321     private void commit(final ActorRef sender, Object serialized) {
322         Modification modification = MutableCompositeModification
323             .fromSerializable(serialized, schemaContext);
324         DOMStoreThreePhaseCommitCohort cohort =
325             modificationToCohort.remove(serialized);
326         if (cohort == null) {
327             LOG.debug(
328                 "Could not find cohort for modification : {}. Writing modification using a new transaction",
329                 modification);
330             DOMStoreWriteTransaction transaction =
331                 store.newWriteOnlyTransaction();
332
333             LOG.debug("Created new transaction {}", transaction.getIdentifier().toString());
334
335             modification.apply(transaction);
336             try {
337                 syncCommitTransaction(transaction);
338             } catch (InterruptedException | ExecutionException e) {
339                 shardMBean.incrementFailedTransactionsCount();
340                 LOG.error("Failed to commit", e);
341                 return;
342             }
343             //we want to just apply the recovery commit and return
344             shardMBean.incrementCommittedTransactionCount();
345             return;
346         }
347
348
349         if(sender == null){
350             LOG.error("Commit failed. Sender cannot be null");
351             return;
352         }
353
354         final ListenableFuture<Void> future = cohort.commit();
355         final ActorRef self = getSelf();
356
357         Futures.addCallback(future, new FutureCallback<Void>() {
358             @Override
359             public void onSuccess(Void v) {
360                 sender.tell(new CommitTransactionReply().toSerializable(), self);
361                 shardMBean.incrementCommittedTransactionCount();
362                 shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
363             }
364
365             @Override
366             public void onFailure(Throwable t) {
367                 LOG.error(t, "An exception happened during commit");
368                 shardMBean.incrementFailedTransactionsCount();
369                 sender.tell(new akka.actor.Status.Failure(t), self);
370             }
371         });
372
373     }
374
375     private void handleForwardedCommit(ForwardedCommitTransaction message) {
376         Object serializedModification =
377             message.getModification().toSerializable();
378
379         modificationToCohort
380             .put(serializedModification, message.getCohort());
381
382         if (persistent) {
383             this.persistData(getSender(), "identifier",
384                 new CompositeModificationPayload(serializedModification));
385         } else {
386             this.commit(getSender(), serializedModification);
387         }
388     }
389
390     private void updateSchemaContext(UpdateSchemaContext message) {
391         this.schemaContext = message.getSchemaContext();
392         updateSchemaContext(message.getSchemaContext());
393         store.onGlobalContextUpdated(message.getSchemaContext());
394     }
395
396     @VisibleForTesting void updateSchemaContext(SchemaContext schemaContext) {
397         store.onGlobalContextUpdated(schemaContext);
398     }
399
400     private void registerChangeListener(
401         RegisterChangeListener registerChangeListener) {
402
403         LOG.debug("registerDataChangeListener for {}", registerChangeListener
404             .getPath());
405
406
407         ActorSelection dataChangeListenerPath = getContext()
408             .system().actorSelection(
409                 registerChangeListener.getDataChangeListenerPath());
410
411
412         // Notify the listener if notifications should be enabled or not
413         // If this shard is the leader then it will enable notifications else
414         // it will not
415         dataChangeListenerPath
416             .tell(new EnableNotification(isLeader()), getSelf());
417
418         // Now store a reference to the data change listener so it can be notified
419         // at a later point if notifications should be enabled or disabled
420         dataChangeListeners.add(dataChangeListenerPath);
421
422         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
423             listener = new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
424
425         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
426             registration = store.registerChangeListener(registerChangeListener.getPath(),
427                 listener, registerChangeListener.getScope());
428         ActorRef listenerRegistration =
429             getContext().actorOf(
430                 DataChangeListenerRegistration.props(registration));
431
432         LOG.debug(
433             "registerDataChangeListener sending reply, listenerRegistrationPath = {} "
434             , listenerRegistration.path().toString());
435
436         getSender()
437             .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
438                 getSelf());
439     }
440
441     private void createTransactionChain() {
442         DOMStoreTransactionChain chain = store.createTransactionChain();
443         ActorRef transactionChain = getContext().actorOf(
444                 ShardTransactionChain.props(chain, schemaContext, datastoreContext, shardMBean));
445         getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
446                 getSelf());
447     }
448
449     private boolean isMetricsCaptureEnabled(){
450         CommonConfig config = new CommonConfig(getContext().system().settings().config());
451         return config.isMetricCaptureEnabled();
452     }
453
454     @Override protected void applyState(ActorRef clientActor, String identifier,
455         Object data) {
456
457         if (data instanceof CompositeModificationPayload) {
458             Object modification =
459                 ((CompositeModificationPayload) data).getModification();
460
461             if (modification != null) {
462                 commit(clientActor, modification);
463             } else {
464                 LOG.error(
465                     "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
466                     identifier, clientActor.path().toString());
467             }
468
469         } else {
470             LOG.error("Unknown state received {}", data);
471         }
472
473         // Update stats
474         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
475
476         if (lastLogEntry != null) {
477             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
478             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
479         }
480
481         shardMBean.setCommitIndex(getCommitIndex());
482         shardMBean.setLastApplied(getLastApplied());
483
484     }
485
486     @Override protected void createSnapshot() {
487         if (createSnapshotTransaction == null) {
488
489             // Create a transaction. We are really going to treat the transaction as a worker
490             // so that this actor does not get block building the snapshot
491             createSnapshotTransaction = createTransaction(
492                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
493                 "createSnapshot", "");
494
495             createSnapshotTransaction.tell(
496                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
497
498         }
499     }
500
501     @VisibleForTesting @Override protected void applySnapshot(ByteString snapshot) {
502         // Since this will be done only on Recovery or when this actor is a Follower
503         // we can safely commit everything in here. We not need to worry about event notifications
504         // as they would have already been disabled on the follower
505         try {
506             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
507             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
508             NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
509                 .decode(YangInstanceIdentifier.builder().build(), serializedNode);
510
511             // delete everything first
512             transaction.delete(YangInstanceIdentifier.builder().build());
513
514             // Add everything from the remote node back
515             transaction.write(YangInstanceIdentifier.builder().build(), node);
516             syncCommitTransaction(transaction);
517         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
518             LOG.error(e, "An exception occurred when applying snapshot");
519         }
520     }
521
522     @Override protected void onStateChanged() {
523         for (ActorSelection dataChangeListener : dataChangeListeners) {
524             dataChangeListener
525                 .tell(new EnableNotification(isLeader()), getSelf());
526         }
527
528         if (getLeaderId() != null) {
529             shardMBean.setLeader(getLeaderId());
530         }
531
532         shardMBean.setRaftState(getRaftState().name());
533         shardMBean.setCurrentTerm(getCurrentTerm());
534
535         // If this actor is no longer the leader close all the transaction chains
536         if(!isLeader()){
537             for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
538                 LOG.debug("onStateChanged: Closing transaction chain {} because shard {} is no longer the leader", entry.getKey(), getId());
539                 entry.getValue().close();
540             }
541
542             transactionChains.clear();
543         }
544     }
545
546     @Override public String persistenceId() {
547         return this.name.toString();
548     }
549
550
551     private static class ShardConfigParams extends DefaultConfigParamsImpl {
552         public static final FiniteDuration HEART_BEAT_INTERVAL =
553             new FiniteDuration(500, TimeUnit.MILLISECONDS);
554
555         @Override public FiniteDuration getHeartBeatInterval() {
556             return HEART_BEAT_INTERVAL;
557         }
558     }
559
560     private static class ShardCreator implements Creator<Shard> {
561
562         private static final long serialVersionUID = 1L;
563
564         final ShardIdentifier name;
565         final Map<ShardIdentifier, String> peerAddresses;
566         final DatastoreContext datastoreContext;
567         final SchemaContext schemaContext;
568
569         ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
570                 DatastoreContext datastoreContext, SchemaContext schemaContext) {
571             this.name = name;
572             this.peerAddresses = peerAddresses;
573             this.datastoreContext = datastoreContext;
574             this.schemaContext = schemaContext;
575         }
576
577         @Override
578         public Shard create() throws Exception {
579             return new Shard(name, peerAddresses, datastoreContext, schemaContext);
580         }
581     }
582
583     @VisibleForTesting NormalizedNode readStore() throws ExecutionException, InterruptedException {
584         DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
585
586         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
587             transaction.read(YangInstanceIdentifier.builder().build());
588
589         NormalizedNode<?, ?> node = future.get().get();
590
591         transaction.close();
592
593         return node;
594     }
595
596     @VisibleForTesting void writeToStore(YangInstanceIdentifier id, NormalizedNode node)
597         throws ExecutionException, InterruptedException {
598         DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
599
600         transaction.write(id, node);
601
602         syncCommitTransaction(transaction);
603     }
604
605 }