atomic-storage: remove type dependency at segment level I/O
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / databroker / ConcurrentDOMDataBrokerTest.java
1 /*
2  * Copyright (c) 2014 Brocade Communications 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.cluster.databroker;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import static org.junit.Assert.assertSame;
13 import static org.junit.Assert.assertTrue;
14 import static org.junit.Assert.fail;
15 import static org.mockito.Mockito.doAnswer;
16 import static org.mockito.Mockito.doReturn;
17 import static org.mockito.Mockito.inOrder;
18 import static org.mockito.Mockito.mock;
19 import static org.mockito.Mockito.never;
20 import static org.mockito.Mockito.times;
21 import static org.mockito.Mockito.verify;
22 import static org.opendaylight.yangtools.util.concurrent.FluentFutures.immediateFalseFluentFuture;
23 import static org.opendaylight.yangtools.util.concurrent.FluentFutures.immediateNullFluentFuture;
24 import static org.opendaylight.yangtools.util.concurrent.FluentFutures.immediateTrueFluentFuture;
25
26 import com.google.common.base.Throwables;
27 import com.google.common.collect.ImmutableMap;
28 import com.google.common.util.concurrent.FluentFuture;
29 import com.google.common.util.concurrent.FutureCallback;
30 import com.google.common.util.concurrent.Futures;
31 import com.google.common.util.concurrent.ListenableFuture;
32 import com.google.common.util.concurrent.MoreExecutors;
33 import com.google.common.util.concurrent.SettableFuture;
34 import com.google.common.util.concurrent.Uninterruptibles;
35 import java.util.ArrayList;
36 import java.util.List;
37 import java.util.concurrent.CountDownLatch;
38 import java.util.concurrent.ExecutionException;
39 import java.util.concurrent.SynchronousQueue;
40 import java.util.concurrent.ThreadPoolExecutor;
41 import java.util.concurrent.TimeUnit;
42 import java.util.concurrent.TimeoutException;
43 import java.util.concurrent.atomic.AtomicReference;
44 import org.junit.After;
45 import org.junit.Before;
46 import org.junit.Test;
47 import org.mockito.InOrder;
48 import org.mockito.stubbing.Answer;
49 import org.opendaylight.mdsal.common.api.CommitInfo;
50 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
51 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
52 import org.opendaylight.mdsal.dom.api.DOMDataBroker.CommitCohortExtension;
53 import org.opendaylight.mdsal.dom.api.DOMDataBroker.DataTreeChangeExtension;
54 import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohort;
55 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
56 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
57 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
58 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
59 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
60 import org.opendaylight.mdsal.dom.spi.TransactionCommitFailedExceptionMapper;
61 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
62 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadTransaction;
63 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction;
64 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
65 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTransactionChain;
66 import org.opendaylight.mdsal.dom.spi.store.DOMStoreWriteTransaction;
67 import org.opendaylight.mdsal.dom.store.inmemory.InMemoryDOMDataStore;
68 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
69 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
70
71 /**
72  * Unit tests for DOMConcurrentDataCommitCoordinator.
73  *
74  * @author Thomas Pantelis
75  */
76 public class ConcurrentDOMDataBrokerTest {
77
78     private final DOMDataTreeWriteTransaction transaction = mock(DOMDataTreeWriteTransaction.class);
79     private final DOMStoreThreePhaseCommitCohort mockCohort = mock(DOMStoreThreePhaseCommitCohort.class);
80     private final ThreadPoolExecutor futureExecutor =
81             new ThreadPoolExecutor(0, 1, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
82     private ConcurrentDOMDataBroker coordinator;
83
84     @Before
85     public void setup() {
86         doReturn("tx").when(transaction).getIdentifier();
87
88         DOMStore store = new InMemoryDOMDataStore("OPER", MoreExecutors.newDirectExecutorService());
89
90         coordinator = new ConcurrentDOMDataBroker(ImmutableMap.of(LogicalDatastoreType.OPERATIONAL, store),
91                 futureExecutor);
92     }
93
94     @After
95     public void tearDown() {
96         futureExecutor.shutdownNow();
97     }
98
99     @Test
100     public void testSuccessfulSubmitAsync() throws Exception {
101         testSuccessfulSubmit(true);
102     }
103
104     @Test
105     public void testSuccessfulSubmitSync() throws Exception {
106         testSuccessfulSubmit(false);
107     }
108
109     private void testSuccessfulSubmit(final boolean doAsync) throws InterruptedException {
110         final CountDownLatch asyncCanCommitContinue = new CountDownLatch(1);
111         Answer<ListenableFuture<Boolean>> asyncCanCommit = invocation -> {
112             final SettableFuture<Boolean> future = SettableFuture.create();
113             if (doAsync) {
114                 new Thread(() -> {
115                     Uninterruptibles.awaitUninterruptibly(asyncCanCommitContinue, 10, TimeUnit.SECONDS);
116                     future.set(Boolean.TRUE);
117                 }).start();
118             } else {
119                 future.set(Boolean.TRUE);
120             }
121
122             return future;
123         };
124
125         doAnswer(asyncCanCommit).when(mockCohort).canCommit();
126         doReturn(immediateNullFluentFuture()).when(mockCohort).preCommit();
127         doReturn(immediateNullFluentFuture()).when(mockCohort).commit();
128
129         ListenableFuture<? extends CommitInfo> future = coordinator.commit(transaction, mockCohort);
130
131         final CountDownLatch doneLatch = new CountDownLatch(1);
132         final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
133         Futures.addCallback(future, new FutureCallback<CommitInfo>() {
134             @Override
135             public void onSuccess(final CommitInfo result) {
136                 doneLatch.countDown();
137             }
138
139             @Override
140             public void onFailure(final Throwable failure) {
141                 caughtEx.set(failure);
142                 doneLatch.countDown();
143             }
144         }, MoreExecutors.directExecutor());
145
146         asyncCanCommitContinue.countDown();
147
148         assertTrue("Submit complete", doneLatch.await(5, TimeUnit.SECONDS));
149
150         if (caughtEx.get() != null) {
151             Throwables.throwIfUnchecked(caughtEx.get());
152             throw new RuntimeException(caughtEx.get());
153         }
154
155         assertEquals("Task count", doAsync ? 1 : 0, futureExecutor.getTaskCount());
156
157         InOrder inOrder = inOrder(mockCohort);
158         inOrder.verify(mockCohort, times(1)).canCommit();
159         inOrder.verify(mockCohort, times(1)).preCommit();
160         inOrder.verify(mockCohort, times(1)).commit();
161     }
162
163     @Test
164     public void testSubmitWithNegativeCanCommitResponse() throws Exception {
165         doReturn(Futures.immediateFuture(Boolean.FALSE)).when(mockCohort).canCommit();
166         doReturn(immediateNullFluentFuture()).when(mockCohort).abort();
167
168         assertFailure(coordinator.commit(transaction, mockCohort), null, mockCohort);
169     }
170
171     private static void assertFailure(final ListenableFuture<?> future, final Exception expCause,
172             final DOMStoreThreePhaseCommitCohort mockCohort) throws Exception {
173         try {
174             future.get(5, TimeUnit.SECONDS);
175             fail("Expected TransactionCommitFailedException");
176         } catch (ExecutionException e) {
177             TransactionCommitFailedException tcf = TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER.apply(e);
178             if (expCause != null) {
179                 assertSame("Expected cause", expCause.getClass(), tcf.getCause().getClass());
180             }
181             verify(mockCohort, times(1)).abort();
182         } catch (TimeoutException e) {
183             throw e;
184         }
185     }
186
187     @Test
188     public void testSubmitWithCanCommitException() throws Exception {
189         final Exception cause = new IllegalStateException("mock");
190         doReturn(Futures.immediateFailedFuture(cause)).when(mockCohort).canCommit();
191         doReturn(immediateNullFluentFuture()).when(mockCohort).abort();
192
193         assertFailure(coordinator.commit(transaction, mockCohort), cause, mockCohort);
194     }
195
196     @Test
197     public void testSubmitWithPreCommitException() throws Exception {
198         doReturn(immediateTrueFluentFuture()).when(mockCohort).canCommit();
199         final IllegalStateException cause = new IllegalStateException("mock");
200         doReturn(Futures.immediateFailedFuture(cause)).when(mockCohort).preCommit();
201         doReturn(immediateNullFluentFuture()).when(mockCohort).abort();
202
203         assertFailure(coordinator.commit(transaction, mockCohort), cause, mockCohort);
204     }
205
206     @Test
207     public void testSubmitWithCommitException() throws Exception {
208         doReturn(immediateTrueFluentFuture()).when(mockCohort).canCommit();
209         doReturn(immediateNullFluentFuture()).when(mockCohort).preCommit();
210         final IllegalStateException cause = new IllegalStateException("mock");
211         doReturn(Futures.immediateFailedFuture(cause)).when(mockCohort).commit();
212         doReturn(immediateNullFluentFuture()).when(mockCohort).abort();
213
214         assertFailure(coordinator.commit(transaction, mockCohort), cause, mockCohort);
215     }
216
217     @Test
218     public void testSubmitWithAbortException() throws Exception {
219         final Exception canCommitCause = new IllegalStateException("canCommit error");
220         doReturn(Futures.immediateFailedFuture(canCommitCause)).when(mockCohort).canCommit();
221         final Exception abortCause = new IllegalStateException("abort error");
222         doReturn(Futures.immediateFailedFuture(abortCause)).when(mockCohort).abort();
223
224         assertFailure(coordinator.commit(transaction, mockCohort), canCommitCause, mockCohort);
225     }
226
227     @Test
228     public void testCreateReadWriteTransaction() {
229         DOMStore domStore = mock(DOMStore.class);
230         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
231                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
232                 futureExecutor)) {
233             dataBroker.newReadWriteTransaction();
234
235             verify(domStore, never()).newReadWriteTransaction();
236         }
237     }
238
239     @Test
240     public void testCreateWriteOnlyTransaction() {
241         DOMStore domStore = mock(DOMStore.class);
242         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
243                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
244                 futureExecutor)) {
245             dataBroker.newWriteOnlyTransaction();
246
247             verify(domStore, never()).newWriteOnlyTransaction();
248         }
249     }
250
251     @Test
252     public void testCreateReadOnlyTransaction() {
253         DOMStore domStore = mock(DOMStore.class);
254         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
255                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
256                 futureExecutor)) {
257             dataBroker.newReadOnlyTransaction();
258
259             verify(domStore, never()).newReadOnlyTransaction();
260         }
261     }
262
263     @Test
264     public void testLazySubTransactionCreationForReadWriteTransactions() {
265         DOMStore configDomStore = mock(DOMStore.class);
266         DOMStore operationalDomStore = mock(DOMStore.class);
267         DOMStoreReadWriteTransaction storeTxn = mock(DOMStoreReadWriteTransaction.class);
268
269         doReturn(storeTxn).when(operationalDomStore).newReadWriteTransaction();
270         doReturn(storeTxn).when(configDomStore).newReadWriteTransaction();
271
272         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
273                 LogicalDatastoreType.OPERATIONAL, operationalDomStore, LogicalDatastoreType.CONFIGURATION,
274                 configDomStore), futureExecutor)) {
275             DOMDataTreeReadWriteTransaction dataTxn = dataBroker.newReadWriteTransaction();
276
277             dataTxn.put(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of(), mock(ContainerNode.class));
278             dataTxn.put(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of(), mock(ContainerNode.class));
279             dataTxn.read(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of());
280
281             verify(configDomStore, never()).newReadWriteTransaction();
282             verify(operationalDomStore, times(1)).newReadWriteTransaction();
283         }
284
285     }
286
287     @Test
288     public void testLazySubTransactionCreationForWriteOnlyTransactions() {
289         DOMStore configDomStore = mock(DOMStore.class);
290         DOMStore operationalDomStore = mock(DOMStore.class);
291         DOMStoreWriteTransaction storeTxn = mock(DOMStoreWriteTransaction.class);
292
293         doReturn(storeTxn).when(operationalDomStore).newWriteOnlyTransaction();
294         doReturn(storeTxn).when(configDomStore).newWriteOnlyTransaction();
295
296         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
297                 LogicalDatastoreType.OPERATIONAL, operationalDomStore, LogicalDatastoreType.CONFIGURATION,
298                 configDomStore), futureExecutor)) {
299             DOMDataTreeWriteTransaction dataTxn = dataBroker.newWriteOnlyTransaction();
300
301             dataTxn.put(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of(), mock(ContainerNode.class));
302             dataTxn.put(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of(), mock(ContainerNode.class));
303
304             verify(configDomStore, never()).newWriteOnlyTransaction();
305             verify(operationalDomStore, times(1)).newWriteOnlyTransaction();
306         }
307     }
308
309     @Test
310     public void testLazySubTransactionCreationForReadOnlyTransactions() {
311         DOMStore configDomStore = mock(DOMStore.class);
312         DOMStore operationalDomStore = mock(DOMStore.class);
313         DOMStoreReadTransaction storeTxn = mock(DOMStoreReadTransaction.class);
314
315         doReturn(storeTxn).when(operationalDomStore).newReadOnlyTransaction();
316         doReturn(storeTxn).when(configDomStore).newReadOnlyTransaction();
317
318         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
319                 LogicalDatastoreType.OPERATIONAL, operationalDomStore, LogicalDatastoreType.CONFIGURATION,
320                 configDomStore), futureExecutor)) {
321             DOMDataTreeReadTransaction dataTxn = dataBroker.newReadOnlyTransaction();
322
323             dataTxn.read(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of());
324             dataTxn.read(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of());
325
326             verify(configDomStore, never()).newReadOnlyTransaction();
327             verify(operationalDomStore, times(1)).newReadOnlyTransaction();
328         }
329     }
330
331     @Test
332     public void testSubmitWithOnlyOneSubTransaction() throws InterruptedException {
333         DOMStore configDomStore = mock(DOMStore.class);
334         DOMStore operationalDomStore = mock(DOMStore.class);
335         DOMStoreReadWriteTransaction mockStoreReadWriteTransaction = mock(DOMStoreReadWriteTransaction.class);
336
337         doReturn(mockStoreReadWriteTransaction).when(operationalDomStore).newReadWriteTransaction();
338         doReturn(mockCohort).when(mockStoreReadWriteTransaction).ready();
339         doReturn(immediateFalseFluentFuture()).when(mockCohort).canCommit();
340         doReturn(immediateNullFluentFuture()).when(mockCohort).abort();
341
342         final CountDownLatch latch = new CountDownLatch(1);
343         final List<DOMStoreThreePhaseCommitCohort> commitCohorts = new ArrayList<>();
344
345         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
346                 LogicalDatastoreType.OPERATIONAL, operationalDomStore, LogicalDatastoreType.CONFIGURATION,
347                 configDomStore), futureExecutor) {
348             @Override
349             public FluentFuture<? extends CommitInfo> commit(DOMDataTreeWriteTransaction writeTx,
350                     DOMStoreThreePhaseCommitCohort cohort) {
351                 commitCohorts.add(cohort);
352                 latch.countDown();
353                 return super.commit(writeTx, cohort);
354             }
355         }) {
356             DOMDataTreeReadWriteTransaction domDataReadWriteTransaction = dataBroker.newReadWriteTransaction();
357
358             domDataReadWriteTransaction.delete(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of());
359
360             domDataReadWriteTransaction.commit();
361
362             assertTrue(latch.await(10, TimeUnit.SECONDS));
363
364             assertTrue(commitCohorts.size() == 1);
365         }
366     }
367
368     @Test
369     public void testCreateTransactionChain() {
370         DOMStore domStore = mock(DOMStore.class);
371         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
372                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
373                 futureExecutor)) {
374
375             dataBroker.createTransactionChain();
376
377             verify(domStore, times(2)).createTransactionChain();
378         }
379
380     }
381
382     @Test
383     public void testCreateTransactionOnChain() {
384         DOMStore domStore = mock(DOMStore.class);
385         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
386                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
387                 futureExecutor)) {
388
389             DOMStoreReadWriteTransaction operationalTransaction = mock(DOMStoreReadWriteTransaction.class);
390             DOMStoreTransactionChain mockChain = mock(DOMStoreTransactionChain.class);
391
392             doReturn(mockChain).when(domStore).createTransactionChain();
393             doReturn(operationalTransaction).when(mockChain).newWriteOnlyTransaction();
394
395             DOMTransactionChain transactionChain = dataBroker.createTransactionChain();
396
397             DOMDataTreeWriteTransaction domDataWriteTransaction = transactionChain.newWriteOnlyTransaction();
398
399             verify(mockChain, never()).newWriteOnlyTransaction();
400
401             domDataWriteTransaction.put(LogicalDatastoreType.OPERATIONAL, YangInstanceIdentifier.of(),
402                     mock(ContainerNode.class));
403         }
404     }
405
406     @Test
407     public void testEmptyTransactionSubmitSucceeds() throws ExecutionException, InterruptedException {
408         DOMStore domStore = mock(DOMStore.class);
409         try (ConcurrentDOMDataBroker dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
410                 LogicalDatastoreType.OPERATIONAL, domStore, LogicalDatastoreType.CONFIGURATION, domStore),
411                 futureExecutor)) {
412
413             FluentFuture<? extends CommitInfo> submit1 = dataBroker.newWriteOnlyTransaction().commit();
414
415             assertNotNull(submit1);
416
417             submit1.get();
418
419             FluentFuture<? extends CommitInfo> submit2 = dataBroker.newReadWriteTransaction().commit();
420
421             assertNotNull(submit2);
422
423             submit2.get();
424         }
425     }
426
427     @Test
428     public void testExtensions() {
429         final var mockConfigStore = mock(ClientBackedDataStore.class);
430         final var mockOperStore = mock(ClientBackedDataStore.class);
431         try (var dataBroker = new ConcurrentDOMDataBroker(ImmutableMap.of(
432                 LogicalDatastoreType.OPERATIONAL, mockOperStore,
433                 LogicalDatastoreType.CONFIGURATION, mockConfigStore), futureExecutor)) {
434             assertNotNull(dataBroker.extension(DataTreeChangeExtension.class));
435
436             final var cohortRegistry = dataBroker.extension(CommitCohortExtension.class);
437             assertNotNull(cohortRegistry);
438
439             final var cohort = mock(DOMDataTreeCommitCohort.class);
440             final var path = DOMDataTreeIdentifier.of(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.of());
441             cohortRegistry.registerCommitCohort(path, cohort);
442
443             verify(mockConfigStore).registerCommitCohort(path, cohort);
444         }
445     }
446 }