Remove remoterpc dead code.
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / store / impl / InMemoryDOMDataStore.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 package org.opendaylight.controller.md.sal.dom.store.impl;
9
10 import static com.google.common.base.Preconditions.checkState;
11
12 import java.util.Collections;
13 import java.util.concurrent.Callable;
14 import java.util.concurrent.atomic.AtomicLong;
15
16 import javax.annotation.concurrent.GuardedBy;
17
18 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
19 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
20 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
21 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
22 import org.opendaylight.controller.md.sal.dom.store.impl.SnapshotBackedWriteTransaction.TransactionReadyPrototype;
23 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ConflictingModificationAppliedException;
24 import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTree;
25 import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeCandidate;
26 import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeModification;
27 import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataTreeSnapshot;
28 import org.opendaylight.controller.md.sal.dom.store.impl.tree.DataValidationFailedException;
29 import org.opendaylight.controller.md.sal.dom.store.impl.tree.ListenerTree;
30 import org.opendaylight.controller.md.sal.dom.store.impl.tree.data.InMemoryDataTreeFactory;
31 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
32 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
33 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
34 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
35 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
36 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
37 import org.opendaylight.yangtools.concepts.AbstractListenerRegistration;
38 import org.opendaylight.yangtools.concepts.Identifiable;
39 import org.opendaylight.yangtools.concepts.ListenerRegistration;
40 import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
41 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
42 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
43 import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
44 import org.slf4j.Logger;
45 import org.slf4j.LoggerFactory;
46
47 import com.google.common.base.Optional;
48 import com.google.common.base.Preconditions;
49 import com.google.common.util.concurrent.FutureCallback;
50 import com.google.common.util.concurrent.Futures;
51 import com.google.common.util.concurrent.ListenableFuture;
52 import com.google.common.util.concurrent.ListeningExecutorService;
53
54 /**
55  * In-memory DOM Data Store
56  *
57  * Implementation of {@link DOMStore} which uses {@link DataTree} and other
58  * classes such as {@link SnapshotBackedWriteTransaction}.
59  * {@link SnapshotBackedReadTransaction} and {@link ResolveDataChangeEventsTask}
60  * to implement {@link DOMStore} contract.
61  *
62  */
63 public class InMemoryDOMDataStore implements DOMStore, Identifiable<String>, SchemaContextListener,
64         TransactionReadyPrototype {
65     private static final Logger LOG = LoggerFactory.getLogger(InMemoryDOMDataStore.class);
66     private final DataTree dataTree = InMemoryDataTreeFactory.getInstance().create();
67     private final ListenerTree listenerTree = ListenerTree.create();
68     private final AtomicLong txCounter = new AtomicLong(0);
69     private final ListeningExecutorService executor;
70     private final String name;
71
72     public InMemoryDOMDataStore(final String name, final ListeningExecutorService executor) {
73         this.name = Preconditions.checkNotNull(name);
74         this.executor = Preconditions.checkNotNull(executor);
75     }
76
77     @Override
78     public final String getIdentifier() {
79         return name;
80     }
81
82     @Override
83     public DOMStoreReadTransaction newReadOnlyTransaction() {
84         return new SnapshotBackedReadTransaction(nextIdentifier(), dataTree.takeSnapshot());
85     }
86
87     @Override
88     public DOMStoreReadWriteTransaction newReadWriteTransaction() {
89         return new SnapshotBackedReadWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
90     }
91
92     @Override
93     public DOMStoreWriteTransaction newWriteOnlyTransaction() {
94         return new SnapshotBackedWriteTransaction(nextIdentifier(), dataTree.takeSnapshot(), this);
95     }
96
97     @Override
98     public DOMStoreTransactionChain createTransactionChain() {
99         return new DOMStoreTransactionChainImpl();
100     }
101
102     @Override
103     public synchronized void onGlobalContextUpdated(final SchemaContext ctx) {
104         dataTree.setSchemaContext(ctx);
105     }
106
107     @Override
108     public <L extends AsyncDataChangeListener<InstanceIdentifier, NormalizedNode<?, ?>>> ListenerRegistration<L> registerChangeListener(
109             final InstanceIdentifier path, final L listener, final DataChangeScope scope) {
110
111         /*
112          * Make sure commit is not occurring right now. Listener has to be
113          * registered and its state capture enqueued at a consistent point.
114          *
115          * FIXME: improve this to read-write lock, such that multiple listener
116          * registrations can occur simultaneously
117          */
118         final DataChangeListenerRegistration<L> reg;
119         synchronized (this) {
120             LOG.debug("{}: Registering data change listener {} for {}", name, listener, path);
121
122             reg = listenerTree.registerDataChangeListener(path, listener, scope);
123
124             Optional<NormalizedNode<?, ?>> currentState = dataTree.takeSnapshot().readNode(path);
125             if (currentState.isPresent()) {
126                 final NormalizedNode<?, ?> data = currentState.get();
127
128                 final DOMImmutableDataChangeEvent event = DOMImmutableDataChangeEvent.builder(DataChangeScope.BASE) //
129                         .setAfter(data) //
130                         .addCreated(path, data) //
131                         .build();
132                 executor.submit(new ChangeListenerNotifyTask(Collections.singletonList(reg), event));
133             }
134         }
135
136         return new AbstractListenerRegistration<L>(listener) {
137             @Override
138             protected void removeRegistration() {
139                 synchronized (InMemoryDOMDataStore.this) {
140                     reg.close();
141                 }
142             }
143         };
144     }
145
146     @Override
147     public synchronized DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction writeTx) {
148         LOG.debug("Tx: {} is submitted. Modifications: {}", writeTx.getIdentifier(), writeTx.getMutatedView());
149         return new ThreePhaseCommitImpl(writeTx);
150     }
151
152     private Object nextIdentifier() {
153         return name + "-" + txCounter.getAndIncrement();
154     }
155
156     private class DOMStoreTransactionChainImpl implements DOMStoreTransactionChain, TransactionReadyPrototype {
157
158         @GuardedBy("this")
159         private SnapshotBackedWriteTransaction latestOutstandingTx;
160
161         private boolean chainFailed = false;
162
163         private void checkFailed() {
164             Preconditions.checkState(!chainFailed, "Transaction chain is failed.");
165         }
166
167         @Override
168         public synchronized DOMStoreReadTransaction newReadOnlyTransaction() {
169             final DataTreeSnapshot snapshot;
170             checkFailed();
171             if (latestOutstandingTx != null) {
172                 checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
173                 snapshot = latestOutstandingTx.getMutatedView();
174             } else {
175                 snapshot = dataTree.takeSnapshot();
176             }
177             return new SnapshotBackedReadTransaction(nextIdentifier(), snapshot);
178         }
179
180         @Override
181         public synchronized DOMStoreReadWriteTransaction newReadWriteTransaction() {
182             final DataTreeSnapshot snapshot;
183             checkFailed();
184             if (latestOutstandingTx != null) {
185                 checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
186                 snapshot = latestOutstandingTx.getMutatedView();
187             } else {
188                 snapshot = dataTree.takeSnapshot();
189             }
190             final SnapshotBackedReadWriteTransaction ret = new SnapshotBackedReadWriteTransaction(nextIdentifier(),
191                     snapshot, this);
192             latestOutstandingTx = ret;
193             return ret;
194         }
195
196         @Override
197         public synchronized DOMStoreWriteTransaction newWriteOnlyTransaction() {
198             final DataTreeSnapshot snapshot;
199             checkFailed();
200             if (latestOutstandingTx != null) {
201                 checkState(latestOutstandingTx.isReady(), "Previous transaction in chain must be ready.");
202                 snapshot = latestOutstandingTx.getMutatedView();
203             } else {
204                 snapshot = dataTree.takeSnapshot();
205             }
206             final SnapshotBackedWriteTransaction ret = new SnapshotBackedWriteTransaction(nextIdentifier(), snapshot,
207                     this);
208             latestOutstandingTx = ret;
209             return ret;
210         }
211
212         @Override
213         public DOMStoreThreePhaseCommitCohort ready(final SnapshotBackedWriteTransaction tx) {
214             DOMStoreThreePhaseCommitCohort storeCohort = InMemoryDOMDataStore.this.ready(tx);
215             return new ChainedTransactionCommitImpl(tx, storeCohort, this);
216         }
217
218         @Override
219         public void close() {
220
221         }
222
223         protected synchronized void onTransactionFailed(final SnapshotBackedWriteTransaction transaction,
224                 final Throwable t) {
225             chainFailed = true;
226
227         }
228
229         public synchronized void onTransactionCommited(final SnapshotBackedWriteTransaction transaction) {
230             // If commited transaction is latestOutstandingTx we clear
231             // latestOutstandingTx
232             // field in order to base new transactions on Datastore Data Tree
233             // directly.
234             if (transaction.equals(latestOutstandingTx)) {
235                 latestOutstandingTx = null;
236             }
237         }
238
239     }
240
241     private static class ChainedTransactionCommitImpl implements DOMStoreThreePhaseCommitCohort {
242
243         private final SnapshotBackedWriteTransaction transaction;
244         private final DOMStoreThreePhaseCommitCohort delegate;
245
246         private final DOMStoreTransactionChainImpl txChain;
247
248         protected ChainedTransactionCommitImpl(final SnapshotBackedWriteTransaction transaction,
249                 final DOMStoreThreePhaseCommitCohort delegate, final DOMStoreTransactionChainImpl txChain) {
250             super();
251             this.transaction = transaction;
252             this.delegate = delegate;
253             this.txChain = txChain;
254         }
255
256         @Override
257         public ListenableFuture<Boolean> canCommit() {
258             return delegate.canCommit();
259         }
260
261         @Override
262         public ListenableFuture<Void> preCommit() {
263             return delegate.preCommit();
264         }
265
266         @Override
267         public ListenableFuture<Void> abort() {
268             return delegate.abort();
269         }
270
271         @Override
272         public ListenableFuture<Void> commit() {
273             ListenableFuture<Void> commitFuture = delegate.commit();
274             Futures.addCallback(commitFuture, new FutureCallback<Void>() {
275                 @Override
276                 public void onFailure(final Throwable t) {
277                     txChain.onTransactionFailed(transaction, t);
278                 }
279
280                 @Override
281                 public void onSuccess(final Void result) {
282                     txChain.onTransactionCommited(transaction);
283                 }
284
285             });
286             return commitFuture;
287         }
288
289     }
290
291     private class ThreePhaseCommitImpl implements DOMStoreThreePhaseCommitCohort {
292
293         private final SnapshotBackedWriteTransaction transaction;
294         private final DataTreeModification modification;
295
296         private ResolveDataChangeEventsTask listenerResolver;
297         private DataTreeCandidate candidate;
298
299         public ThreePhaseCommitImpl(final SnapshotBackedWriteTransaction writeTransaction) {
300             this.transaction = writeTransaction;
301             this.modification = transaction.getMutatedView();
302         }
303
304         @Override
305         public ListenableFuture<Boolean> canCommit() {
306             return executor.submit(new Callable<Boolean>() {
307                 @Override
308                 public Boolean call() throws TransactionCommitFailedException {
309                     try {
310                         dataTree.validate(modification);
311                         LOG.debug("Store Transaction: {} can be committed", transaction.getIdentifier());
312                         return true;
313                     } catch (ConflictingModificationAppliedException e) {
314                         LOG.warn("Store Tx: {} Conflicting modification for {}.", transaction.getIdentifier(),
315                                 e.getPath());
316                         throw new OptimisticLockFailedException("Optimistic lock failed.",e);
317                     } catch (DataValidationFailedException e) {
318                         LOG.warn("Store Tx: {} Data Precondition failed for {}.", transaction.getIdentifier(),
319                                 e.getPath(), e);
320                         throw new TransactionCommitFailedException("Data did not pass validation.",e);
321                     }
322                 }
323             });
324         }
325
326         @Override
327         public ListenableFuture<Void> preCommit() {
328             return executor.submit(new Callable<Void>() {
329                 @Override
330                 public Void call() {
331                     candidate = dataTree.prepare(modification);
332                     listenerResolver = ResolveDataChangeEventsTask.create(candidate, listenerTree);
333                     return null;
334                 }
335             });
336         }
337
338         @Override
339         public ListenableFuture<Void> abort() {
340             candidate = null;
341             return Futures.immediateFuture(null);
342         }
343
344         @Override
345         public ListenableFuture<Void> commit() {
346             checkState(candidate != null, "Proposed subtree must be computed");
347
348             /*
349              * The commit has to occur atomically with regard to listener
350              * registrations.
351              */
352             synchronized (this) {
353                 dataTree.commit(candidate);
354
355                 for (ChangeListenerNotifyTask task : listenerResolver.call()) {
356                     LOG.trace("Scheduling invocation of listeners: {}", task);
357                     executor.submit(task);
358                 }
359             }
360
361             return Futures.immediateFuture(null);
362         }
363     }
364 }