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