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