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