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