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