Merge "Fixed namespace of yang testing file."
[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.util.concurrent.ListenableFuture;
20 import com.google.common.util.concurrent.ListeningExecutorService;
21 import com.google.common.util.concurrent.MoreExecutors;
22
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.RaftActor;
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.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
48 import java.util.ArrayList;
49 import java.util.HashMap;
50 import java.util.List;
51 import java.util.Map;
52 import java.util.concurrent.ExecutionException;
53 import java.util.concurrent.Executors;
54
55 /**
56  * A Shard represents a portion of the logical data tree <br/>
57  * <p>
58  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
59  * </p>
60  */
61 public class Shard extends RaftActor {
62
63     public static final String DEFAULT_NAME = "default";
64
65     private final ListeningExecutorService storeExecutor =
66         MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(2));
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);
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 = new InMemoryDOMDataStore(name, storeExecutor);
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.getClass().equals(RegisterChangeListener.SERIALIZABLE_CLASS)) {
127             registerChangeListener(RegisterChangeListener.fromSerializable(getContext().system(), 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 void createTransaction(CreateTransaction createTransaction) {
147         DOMStoreReadWriteTransaction transaction =
148             store.newReadWriteTransaction();
149         String transactionId = "shard-" + createTransaction.getTransactionId();
150         LOG.info("Creating transaction : {} " , transactionId);
151         ActorRef transactionActor = getContext().actorOf(
152             ShardTransaction.props(transaction, getSelf(), schemaContext), transactionId);
153
154         getSender()
155             .tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor), createTransaction.getTransactionId()).toSerializable(),
156                 getSelf());
157     }
158
159     private void commit(final ActorRef sender, Object serialized) {
160         Modification modification = MutableCompositeModification.fromSerializable(serialized, schemaContext);
161         DOMStoreThreePhaseCommitCohort cohort =
162             modificationToCohort.remove(serialized);
163         if (cohort == null) {
164             LOG.error(
165                 "Could not find cohort for modification : {}", modification);
166             LOG.info("Writing modification using a new transaction");
167             DOMStoreReadWriteTransaction transaction =
168                 store.newReadWriteTransaction();
169             modification.apply(transaction);
170             DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
171             ListenableFuture<Void> future =
172                 commitCohort.preCommit();
173             try {
174                 future.get();
175                 future = commitCohort.commit();
176                 future.get();
177             } catch (InterruptedException e) {
178                 LOG.error("Failed to commit", e);
179             } catch (ExecutionException e) {
180                 LOG.error("Failed to commit", e);
181             }
182         }
183
184         final ListenableFuture<Void> future = cohort.commit();
185         shardMBean.incrementCommittedTransactionCount();
186         final ActorRef self = getSelf();
187         future.addListener(new Runnable() {
188             @Override
189             public void run() {
190                 try {
191                     future.get();
192
193                     if(sender != null) {
194                         sender
195                             .tell(new CommitTransactionReply().toSerializable(),
196                                 self);
197                     } else {
198                         LOG.error("sender is null ???");
199                     }
200                 } catch (InterruptedException | ExecutionException e) {
201                     // FIXME : Handle this properly
202                     LOG.error(e, "An exception happened when committing");
203                 }
204             }
205         }, getContext().dispatcher());
206     }
207
208     private void handleForwardedCommit(ForwardedCommitTransaction message) {
209         Object serializedModification = message.getModification().toSerializable();
210
211         modificationToCohort
212             .put(serializedModification , message.getCohort());
213
214         if(persistent) {
215             this.persistData(getSender(), "identifier", new CompositeModificationPayload(serializedModification));
216         } else {
217             this.commit(getSender(), serializedModification);
218         }
219     }
220
221     private void updateSchemaContext(UpdateSchemaContext message) {
222         this.schemaContext = message.getSchemaContext();
223         store.onGlobalContextUpdated(message.getSchemaContext());
224     }
225
226     private void registerChangeListener(
227         RegisterChangeListener registerChangeListener) {
228
229         LOG.debug("registerDataChangeListener for " + registerChangeListener.getPath());
230
231
232         ActorSelection dataChangeListenerPath = getContext()
233             .system().actorSelection(
234                 registerChangeListener.getDataChangeListenerPath());
235
236
237         // Notify the listener if notifications should be enabled or not
238         // If this shard is the leader then it will enable notifications else
239         // it will not
240         dataChangeListenerPath.tell(new EnableNotification(isLeader()), getSelf());
241
242         // Now store a reference to the data change listener so it can be notified
243         // at a later point if notifications should be enabled or disabled
244         dataChangeListeners.add(dataChangeListenerPath);
245
246         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
247             listener = new DataChangeListenerProxy(schemaContext,dataChangeListenerPath);
248
249         org.opendaylight.yangtools.concepts.ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
250             registration =
251             store.registerChangeListener(registerChangeListener.getPath(),
252                 listener, registerChangeListener.getScope());
253         ActorRef listenerRegistration =
254             getContext().actorOf(
255                 DataChangeListenerRegistration.props(registration));
256
257         LOG.debug("registerDataChangeListener sending reply, listenerRegistrationPath = " + listenerRegistration.path().toString());
258
259         getSender()
260             .tell(new RegisterChangeListenerReply(listenerRegistration.path()).toSerializable(),
261                 getSelf());
262     }
263
264     private void createTransactionChain() {
265         DOMStoreTransactionChain chain = store.createTransactionChain();
266         ActorRef transactionChain =
267             getContext().actorOf(
268                 ShardTransactionChain.props(chain, schemaContext));
269         getSender()
270             .tell(new CreateTransactionChainReply(transactionChain.path())
271                 .toSerializable(),
272                 getSelf());
273     }
274
275     @Override protected void applyState(ActorRef clientActor, String identifier,
276         Object data) {
277
278         if(data instanceof CompositeModificationPayload){
279             Object modification =
280                 ((CompositeModificationPayload) data).getModification();
281
282             if(modification != null){
283                 commit(clientActor, modification);
284             } else {
285                 LOG.error("modification is null - this is very unexpected");
286             }
287
288
289         } else {
290             LOG.error("Unknown state received {}", data);
291         }
292
293     }
294
295     @Override protected Object createSnapshot() {
296         throw new UnsupportedOperationException("createSnapshot");
297     }
298
299     @Override protected void applySnapshot(Object snapshot) {
300         throw new UnsupportedOperationException("applySnapshot");
301     }
302
303     @Override protected void onStateChanged() {
304         for(ActorSelection dataChangeListener : dataChangeListeners){
305             dataChangeListener.tell(new EnableNotification(isLeader()), getSelf());
306         }
307     }
308
309     @Override public String persistenceId() {
310         return this.name;
311     }
312 }