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