Merge "Fix for possible NPE if Bundle is stopped."
[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.Props;
14 import akka.event.Logging;
15 import akka.event.LoggingAdapter;
16 import akka.japi.Creator;
17 import akka.serialization.Serialization;
18 import com.google.common.base.Optional;
19 import com.google.common.base.Preconditions;
20 import com.google.common.util.concurrent.FutureCallback;
21 import com.google.common.util.concurrent.Futures;
22 import com.google.common.util.concurrent.ListenableFuture;
23 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
24 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
25 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
26 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
27 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
28 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
29 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChain;
30 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionChainReply;
31 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
32 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
33 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
34 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
35 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
36 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
37 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
38 import org.opendaylight.controller.cluster.datastore.modification.Modification;
39 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
40 import org.opendaylight.controller.cluster.raft.ConfigParams;
41 import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
42 import org.opendaylight.controller.cluster.raft.RaftActor;
43 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
44 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
45 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
46 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreConfigProperties;
47 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
48 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
49 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
50 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
51 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
52 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
53 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
54 import scala.concurrent.duration.FiniteDuration;
55
56 import java.util.ArrayList;
57 import java.util.Date;
58 import java.util.HashMap;
59 import java.util.List;
60 import java.util.Map;
61 import java.util.concurrent.ExecutionException;
62 import java.util.concurrent.TimeUnit;
63
64 /**
65  * A Shard represents a portion of the logical data tree <br/>
66  * <p>
67  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
68  * </p>
69  */
70 public class Shard extends RaftActor {
71
72     private static final ConfigParams configParams = new ShardConfigParams();
73
74     public static final String DEFAULT_NAME = "default";
75
76     // The state of this Shard
77     private final InMemoryDOMDataStore store;
78
79     private final Map<Object, DOMStoreThreePhaseCommitCohort>
80         modificationToCohort = new HashMap<>();
81
82     private final LoggingAdapter LOG =
83         Logging.getLogger(getContext().system(), this);
84
85     // By default persistent will be true and can be turned off using the system
86     // property shard.persistent
87     private final boolean persistent;
88
89     /// The name of this shard
90     private final ShardIdentifier name;
91
92     private volatile SchemaContext schemaContext;
93
94     private final ShardStats shardMBean;
95
96     private final List<ActorSelection> dataChangeListeners = new ArrayList<>();
97
98     private Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
99             InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
100         super(name.toString(), mapPeerAddresses(peerAddresses), Optional.of(configParams));
101
102         this.name = name;
103
104         String setting = System.getProperty("shard.persistent");
105
106         this.persistent = !"false".equals(setting);
107
108         LOG.info("Shard created : {} persistent : {}", name, persistent);
109
110         store = InMemoryDOMDataStoreFactory.create(name.toString(), null, dataStoreProperties);
111
112         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString());
113
114     }
115
116     private static Map<String, String> mapPeerAddresses(
117         Map<ShardIdentifier, String> peerAddresses) {
118         Map<String, String> map = new HashMap<>();
119
120         for (Map.Entry<ShardIdentifier, String> entry : peerAddresses
121             .entrySet()) {
122             map.put(entry.getKey().toString(), entry.getValue());
123         }
124
125         return map;
126     }
127
128
129
130
131     public static Props props(final ShardIdentifier name,
132         final Map<ShardIdentifier, String> peerAddresses,
133         final InMemoryDOMDataStoreConfigProperties dataStoreProperties) {
134         Preconditions.checkNotNull(name, "name should not be null");
135         Preconditions
136             .checkNotNull(peerAddresses, "peerAddresses should not be null");
137
138         return Props.create(new Creator<Shard>() {
139
140             @Override
141             public Shard create() throws Exception {
142                 return new Shard(name, peerAddresses, dataStoreProperties);
143             }
144
145         });
146     }
147
148
149     @Override public void onReceiveCommand(Object message) {
150         LOG.debug("Received message {} from {}", message.getClass().toString(),
151             getSender());
152
153         if (message.getClass()
154             .equals(CreateTransactionChain.SERIALIZABLE_CLASS)) {
155             if (isLeader()) {
156                 createTransactionChain();
157             } else if (getLeader() != null) {
158                 getLeader().forward(message, getContext());
159             }
160         } else if (message instanceof RegisterChangeListener) {
161             registerChangeListener((RegisterChangeListener) message);
162         } else if (message instanceof UpdateSchemaContext) {
163             updateSchemaContext((UpdateSchemaContext) message);
164         } else if (message instanceof ForwardedCommitTransaction) {
165             handleForwardedCommit((ForwardedCommitTransaction) message);
166         } else if (message.getClass()
167             .equals(CreateTransaction.SERIALIZABLE_CLASS)) {
168             if (isLeader()) {
169                 createTransaction(CreateTransaction.fromSerializable(message));
170             } else if (getLeader() != null) {
171                 getLeader().forward(message, getContext());
172             }
173         } else if (message instanceof PeerAddressResolved) {
174             PeerAddressResolved resolved = (PeerAddressResolved) message;
175             setPeerAddress(resolved.getPeerId().toString(),
176                 resolved.getPeerAddress());
177         } else {
178             super.onReceiveCommand(message);
179         }
180     }
181
182     private ActorRef createTypedTransactionActor(
183         CreateTransaction createTransaction,
184         ShardTransactionIdentifier transactionId) {
185         if (createTransaction.getTransactionType()
186             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
187
188             shardMBean.incrementReadOnlyTransactionCount();
189
190             return getContext().actorOf(
191                 ShardTransaction
192                     .props(store.newReadOnlyTransaction(), getSelf(),
193                         schemaContext), transactionId.toString());
194
195         } else if (createTransaction.getTransactionType()
196             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
197
198             shardMBean.incrementReadWriteTransactionCount();
199
200             return getContext().actorOf(
201                 ShardTransaction
202                     .props(store.newReadWriteTransaction(), getSelf(),
203                         schemaContext), transactionId.toString());
204
205
206         } else if (createTransaction.getTransactionType()
207             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
208
209             shardMBean.incrementWriteOnlyTransactionCount();
210
211             return getContext().actorOf(
212                 ShardTransaction
213                     .props(store.newWriteOnlyTransaction(), getSelf(),
214                         schemaContext), transactionId.toString());
215         } else {
216             throw new IllegalArgumentException(
217                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
218                     + createTransaction.getTransactionType());
219         }
220     }
221
222     private void createTransaction(CreateTransaction createTransaction) {
223
224         ShardTransactionIdentifier transactionId =
225             ShardTransactionIdentifier.builder()
226                 .remoteTransactionId(createTransaction.getTransactionId())
227                 .build();
228         LOG.debug("Creating transaction : {} ", transactionId);
229         ActorRef transactionActor =
230             createTypedTransactionActor(createTransaction, transactionId);
231
232         getSender()
233             .tell(new CreateTransactionReply(
234                     Serialization.serializedActorPath(transactionActor),
235                     createTransaction.getTransactionId()).toSerializable(),
236                 getSelf());
237     }
238
239     private void commit(final ActorRef sender, Object serialized) {
240         Modification modification = MutableCompositeModification
241             .fromSerializable(serialized, schemaContext);
242         DOMStoreThreePhaseCommitCohort cohort =
243             modificationToCohort.remove(serialized);
244         if (cohort == null) {
245             LOG.debug(
246                 "Could not find cohort for modification : {}. Writing modification using a new transaction",
247                 modification);
248             DOMStoreReadWriteTransaction transaction =
249                 store.newReadWriteTransaction();
250             modification.apply(transaction);
251             DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
252             ListenableFuture<Void> future =
253                 commitCohort.preCommit();
254             try {
255                 future.get();
256                 future = commitCohort.commit();
257                 future.get();
258             } catch (InterruptedException | ExecutionException e) {
259                 shardMBean.incrementFailedTransactionsCount();
260                 LOG.error("Failed to commit", e);
261                 return;
262             }
263             //we want to just apply the recovery commit and return
264             shardMBean.incrementCommittedTransactionCount();
265             return;
266         }
267
268         final ListenableFuture<Void> future = cohort.commit();
269         final ActorRef self = getSelf();
270
271         Futures.addCallback(future, new FutureCallback<Void>() {
272             public void onSuccess(Void v) {
273                sender.tell(new CommitTransactionReply().toSerializable(),self);
274                shardMBean.incrementCommittedTransactionCount();
275                shardMBean.setLastCommittedTransactionTime(new Date());
276             }
277
278             public void onFailure(Throwable t) {
279                 LOG.error(t, "An exception happened during commit");
280                 shardMBean.incrementFailedTransactionsCount();
281                 sender.tell(new akka.actor.Status.Failure(t), self);
282             }
283         });
284
285     }
286
287     private void handleForwardedCommit(ForwardedCommitTransaction message) {
288         Object serializedModification =
289             message.getModification().toSerializable();
290
291         modificationToCohort
292             .put(serializedModification, message.getCohort());
293
294         if (persistent) {
295             this.persistData(getSender(), "identifier",
296                 new CompositeModificationPayload(serializedModification));
297         } else {
298             this.commit(getSender(), serializedModification);
299         }
300     }
301
302     private void updateSchemaContext(UpdateSchemaContext message) {
303         this.schemaContext = message.getSchemaContext();
304         store.onGlobalContextUpdated(message.getSchemaContext());
305     }
306
307     private void registerChangeListener(
308         RegisterChangeListener registerChangeListener) {
309
310         LOG.debug("registerDataChangeListener for {}", registerChangeListener
311             .getPath());
312
313
314         ActorSelection dataChangeListenerPath = getContext()
315             .system().actorSelection(
316                 registerChangeListener.getDataChangeListenerPath());
317
318
319         // Notify the listener if notifications should be enabled or not
320         // If this shard is the leader then it will enable notifications else
321         // it will not
322         dataChangeListenerPath
323             .tell(new EnableNotification(isLeader()), getSelf());
324
325         // Now store a reference to the data change listener so it can be notified
326         // at a later point if notifications should be enabled or disabled
327         dataChangeListeners.add(dataChangeListenerPath);
328
329         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
330             listener =
331             new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
332
333         org.opendaylight.yangtools.concepts.ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
334             registration =
335             store.registerChangeListener(registerChangeListener.getPath(),
336                 listener, registerChangeListener.getScope());
337         ActorRef listenerRegistration =
338             getContext().actorOf(
339                 DataChangeListenerRegistration.props(registration));
340
341         LOG.debug(
342             "registerDataChangeListener sending reply, listenerRegistrationPath = {} "
343             , listenerRegistration.path().toString());
344
345         getSender()
346             .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
347                 getSelf());
348     }
349
350     private void createTransactionChain() {
351         DOMStoreTransactionChain chain = store.createTransactionChain();
352         ActorRef transactionChain =
353             getContext().actorOf(
354                 ShardTransactionChain.props(chain, schemaContext));
355         getSender()
356             .tell(new CreateTransactionChainReply(transactionChain.path())
357                     .toSerializable(),
358                 getSelf());
359     }
360
361     @Override protected void applyState(ActorRef clientActor, String identifier,
362         Object data) {
363
364         if (data instanceof CompositeModificationPayload) {
365             Object modification =
366                 ((CompositeModificationPayload) data).getModification();
367
368             if (modification != null) {
369                 commit(clientActor, modification);
370             } else {
371                 LOG.error(
372                     "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
373                     identifier, clientActor.path().toString());
374             }
375
376
377         } else {
378             LOG.error("Unknown state received {}", data);
379         }
380
381         // Update stats
382         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
383
384         if (lastLogEntry != null) {
385             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
386             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
387         }
388
389         shardMBean.setCommitIndex(getCommitIndex());
390         shardMBean.setLastApplied(getLastApplied());
391
392     }
393
394     @Override protected Object createSnapshot() {
395         throw new UnsupportedOperationException("createSnapshot");
396     }
397
398     @Override protected void applySnapshot(Object snapshot) {
399         throw new UnsupportedOperationException("applySnapshot");
400     }
401
402     @Override protected void onStateChanged() {
403         for (ActorSelection dataChangeListener : dataChangeListeners) {
404             dataChangeListener
405                 .tell(new EnableNotification(isLeader()), getSelf());
406         }
407
408         if (getLeaderId() != null) {
409             shardMBean.setLeader(getLeaderId());
410         }
411
412         shardMBean.setRaftState(getRaftState().name());
413         shardMBean.setCurrentTerm(getCurrentTerm());
414     }
415
416     @Override public String persistenceId() {
417         return this.name.toString();
418     }
419
420
421     private static class ShardConfigParams extends DefaultConfigParamsImpl {
422         public static final FiniteDuration HEART_BEAT_INTERVAL =
423             new FiniteDuration(500, TimeUnit.MILLISECONDS);
424
425         @Override public FiniteDuration getHeartBeatInterval() {
426             return HEART_BEAT_INTERVAL;
427         }
428     }
429 }