Move AbstractDOMDataBroker to mdsal-dom-spi
[mdsal.git] / dom / mdsal-dom-broker / src / test / java / org / opendaylight / mdsal / dom / broker / DOMBrokerTest.java
index 53e5643726998ccd1088c1725321ab743c4ee419..243b03d8c66561961e23146bc403daddd1e3ffb2 100644 (file)
@@ -14,13 +14,12 @@ import static org.junit.Assert.assertTrue;
 import static org.opendaylight.mdsal.common.api.LogicalDatastoreType.CONFIGURATION;
 import static org.opendaylight.mdsal.common.api.LogicalDatastoreType.OPERATIONAL;
 
-import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ForwardingExecutorService;
-import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import java.util.Collections;
+import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -37,36 +36,32 @@ import org.opendaylight.mdsal.common.api.TransactionCommitDeadlockException;
 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
-import org.opendaylight.mdsal.dom.broker.util.TestModel;
+import org.opendaylight.mdsal.dom.spi.AbstractDOMDataBroker;
 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
 import org.opendaylight.mdsal.dom.store.inmemory.InMemoryDOMDataStore;
 import org.opendaylight.yangtools.util.concurrent.DeadlockDetectingListeningExecutorService;
 import org.opendaylight.yangtools.util.concurrent.SpecialExecutors;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
-import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
-import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.data.spi.node.ImmutableNodes;
 
-public class DOMBrokerTest {
+public class DOMBrokerTest extends AbstractDatastoreTest {
 
-    private SchemaContext schemaContext;
     private AbstractDOMDataBroker domBroker;
     private ListeningExecutorService executor;
     private ExecutorService futureExecutor;
     private CommitExecutorService commitExecutor;
 
     @Before
-    public void setupStore() throws Exception {
+    public void setupStore() {
         final InMemoryDOMDataStore operStore = new InMemoryDOMDataStore("OPER",
                 MoreExecutors.newDirectExecutorService());
         final InMemoryDOMDataStore configStore = new InMemoryDOMDataStore("CFG",
                 MoreExecutors.newDirectExecutorService());
-        schemaContext = TestModel.createTestContext();
 
-        operStore.onGlobalContextUpdated(schemaContext);
-        configStore.onGlobalContextUpdated(schemaContext);
+        operStore.onModelContextUpdated(SCHEMA_CONTEXT);
+        configStore.onModelContextUpdated(SCHEMA_CONTEXT);
 
         final ImmutableMap<LogicalDatastoreType, DOMStore> stores =
                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
@@ -75,7 +70,7 @@ public class DOMBrokerTest {
                 .build();
 
         commitExecutor = new CommitExecutorService(Executors.newSingleThreadExecutor());
-        futureExecutor = SpecialExecutors.newBlockingBoundedCachedThreadPool(1, 5, "FCB");
+        futureExecutor = SpecialExecutors.newBlockingBoundedCachedThreadPool(1, 5, "FCB", DOMBrokerTest.class);
         executor = new DeadlockDetectingListeningExecutorService(commitExecutor,
                 TransactionCommitDeadlockException.DEADLOCK_EXCEPTION_SUPPLIER, futureExecutor);
         domBroker = new SerializedDOMDataBroker(stores, executor);
@@ -106,15 +101,16 @@ public class DOMBrokerTest {
          * Writes /test in writeTx.
          *
          */
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
 
         /**
          * Reads /test from readTx Read should return Absent.
          *
          */
-        final ListenableFuture<Optional<NormalizedNode<?, ?>>> readTxContainer = readTx
-                .read(OPERATIONAL, TestModel.TEST_PATH);
-        assertFalse(readTxContainer.get().isPresent());
+        final var readTxContainer = readTx.read(OPERATIONAL, TestModel.TEST_PATH);
+        assertEquals(Optional.empty(), readTxContainer.get());
     }
 
     @Test(timeout = 10000)
@@ -125,58 +121,63 @@ public class DOMBrokerTest {
          * Writes /test in writeTx
          *
          */
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
 
-        writeTx.submit().get();
+        writeTx.commit().get();
 
-        final Optional<NormalizedNode<?, ?>> afterCommitRead = domBroker.newReadOnlyTransaction()
-                .read(OPERATIONAL, TestModel.TEST_PATH).get();
+        final var afterCommitRead = domBroker.newReadOnlyTransaction().read(OPERATIONAL, TestModel.TEST_PATH).get();
         assertTrue(afterCommitRead.isPresent());
     }
 
     @Test(expected = TransactionCommitFailedException.class)
-    public void testRejectedCommit() throws Exception {
+    @SuppressWarnings({"checkstyle:AvoidHidingCauseException", "checkstyle:IllegalThrows"})
+    public void testRejectedCommit() throws Throwable {
         commitExecutor.delegate = Mockito.mock(ExecutorService.class);
         Mockito.doThrow(new RejectedExecutionException("mock"))
             .when(commitExecutor.delegate).execute(Mockito.any(Runnable.class));
         Mockito.doNothing().when(commitExecutor.delegate).shutdown();
         Mockito.doReturn(Collections.emptyList()).when(commitExecutor.delegate).shutdownNow();
         Mockito.doReturn("").when(commitExecutor.delegate).toString();
-        Mockito.doReturn(true).when(commitExecutor.delegate)
+        Mockito.doReturn(Boolean.TRUE).when(commitExecutor.delegate)
             .awaitTermination(Mockito.anyLong(), Mockito.any(TimeUnit.class));
 
         final DOMDataTreeWriteTransaction writeTx = domBroker.newWriteOnlyTransaction();
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
 
-        writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
+        try {
+            writeTx.commit().get(5, TimeUnit.SECONDS);
+        } catch (ExecutionException e) {
+            throw e.getCause();
+        }
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
     AtomicReference<Throwable> submitTxAsync(final DOMDataTreeWriteTransaction writeTx) {
         final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
-        new Thread() {
-            @Override
-            public void run() {
-
-                try {
-                    writeTx.submit();
-                } catch (final Throwable e) {
-                    caughtEx.set(e);
-                }
+        new Thread(() -> {
+            try {
+                writeTx.commit();
+            } catch (final Throwable e) {
+                caughtEx.set(e);
             }
-
-        }.start();
+        }).start();
 
         return caughtEx;
     }
 
     @Test(expected = ReadFailedException.class)
-    public void basicTests() throws Exception {
-        final DataContainerChild<?, ?> outerList = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
-                .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1))
-                .build();
-        final NormalizedNode<?, ?> testContainer = Builders.containerBuilder()
-                .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+    @SuppressWarnings({"checkstyle:IllegalThrows", "checkstyle:AvoidHidingCauseException"})
+    public void basicTests() throws Throwable {
+        final DataContainerChild outerList = ImmutableNodes.newSystemMapBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.OUTER_LIST_QNAME))
+            .withChild(TestUtils.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1))
+            .build();
+        final ContainerNode testContainer = ImmutableNodes.newContainerBuilder()
+                .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
                 .withChild(outerList)
                 .build();
 
@@ -186,32 +187,45 @@ public class DOMBrokerTest {
         assertNotNull(readRx);
         assertNotNull(((SerializedDOMDataBroker) domBroker).getCommitStatsTracker());
 
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-        writeTx.submit().get();
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
+        writeTx.commit().get();
         assertFalse(writeTx.cancel());
 
-        assertEquals(false, domBroker.newReadOnlyTransaction().exists(CONFIGURATION, TestModel.TEST_PATH).get());
-        assertEquals(true, domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
-        assertEquals(false, domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST2_PATH).get());
+        assertFalse(domBroker.newReadOnlyTransaction().exists(CONFIGURATION, TestModel.TEST_PATH).get());
+        assertTrue(domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
+        assertFalse(domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST2_PATH).get());
 
         writeTx = domBroker.newWriteOnlyTransaction();
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
         writeTx.delete(OPERATIONAL, TestModel.TEST_PATH);
-        writeTx.submit().get();
-        assertEquals(false, domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
+        writeTx.commit().get();
+        assertFalse(domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
         assertTrue(domBroker.newWriteOnlyTransaction().cancel());
 
         writeTx = domBroker.newWriteOnlyTransaction();
-        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        writeTx.put(OPERATIONAL, TestModel.TEST_PATH, ImmutableNodes.newContainerBuilder()
+            .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+            .build());
         writeTx.merge(OPERATIONAL, TestModel.TEST_PATH, testContainer);
-        writeTx.submit().get();
-        assertEquals(true, domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
-        assertEquals(true, domBroker.newReadOnlyTransaction().read(OPERATIONAL, TestModel.TEST_PATH).get()
-                 .get().toString().contains(testContainer.toString()));
+        writeTx.commit().get();
+        assertTrue(domBroker.newReadOnlyTransaction().exists(OPERATIONAL, TestModel.TEST_PATH).get());
+        assertTrue(domBroker.newReadOnlyTransaction().read(OPERATIONAL, TestModel.TEST_PATH).get()
+                 .orElseThrow().toString().contains(testContainer.toString()));
 
+        readRx.read(OPERATIONAL, TestModel.TEST_PATH).get(); // init backing tx before close
         readRx.close();
+
         //Expected exception after close call
-        readRx.read(OPERATIONAL, TestModel.TEST_PATH).checkedGet();
+
+        try {
+            readRx.read(OPERATIONAL, TestModel.TEST_PATH).get();
+        } catch (ExecutionException e) {
+            throw e.getCause();
+        }
     }
 
     @SuppressWarnings({"checkstyle:IllegalThrows", "checkstyle:IllegalCatch"})