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