Eliminate use of deprecated mockito methods
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / TransactionProxyTest.java
index 5fbf87827c76ae1c60546028757394089c398858..dd0c9879db71803a31007dfa2164f900ba84b3cc 100644 (file)
@@ -9,12 +9,13 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isA;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -29,21 +30,24 @@ import akka.actor.ActorSystem;
 import akka.actor.Props;
 import akka.dispatch.Futures;
 import akka.util.Timeout;
-import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.FluentFuture;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.util.Collection;
 import java.util.List;
+import java.util.Optional;
+import java.util.SortedSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 import org.mockito.InOrder;
 import org.mockito.Mockito;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
@@ -57,6 +61,7 @@ import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
+import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
@@ -78,7 +83,7 @@ import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import scala.concurrent.Promise;
 
-@SuppressWarnings("resource")
+@SuppressWarnings({"resource", "checkstyle:IllegalThrows", "checkstyle:AvoidHidingCauseException"})
 public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
     @SuppressWarnings("serial")
@@ -86,7 +91,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     interface Invoker {
-        CheckedFuture<?, ReadFailedException> invoke(TransactionProxy proxy) throws Exception;
+        FluentFuture<?> invoke(TransactionProxy proxy);
     }
 
     @Test
@@ -101,7 +106,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
                 TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
 
-        assertEquals("NormalizedNode isPresent", false, readOptional.isPresent());
+        assertFalse("NormalizedNode isPresent", readOptional.isPresent());
 
         NormalizedNode<?, ?> expectedNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
 
@@ -110,13 +115,13 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         readOptional = transactionProxy.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
 
-        assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
+        assertTrue("NormalizedNode isPresent", readOptional.isPresent());
 
         assertEquals("Response NormalizedNode", expectedNode, readOptional.get());
     }
 
     @Test(expected = ReadFailedException.class)
-    public void testReadWithInvalidReplyMessageType() throws Exception {
+    public void testReadWithInvalidReplyMessageType() throws Throwable {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
 
         doReturn(Futures.successful(new Object())).when(mockActorContext)
@@ -124,11 +129,15 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         TransactionProxy transactionProxy = new TransactionProxy(mockComponentFactory, READ_ONLY);
 
-        transactionProxy.read(TestModel.TEST_PATH).checkedGet(5, TimeUnit.SECONDS);
+        try {
+            transactionProxy.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
+        } catch (ExecutionException e) {
+            throw e.getCause();
+        }
     }
 
     @Test(expected = TestException.class)
-    public void testReadWithAsyncRemoteOperatonFailure() throws Exception {
+    public void testReadWithAsyncRemoteOperatonFailure() throws Throwable {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
 
         doReturn(Futures.failed(new TestException())).when(mockActorContext)
@@ -140,7 +149,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     private void testExceptionOnInitialCreateTransaction(final Exception exToThrow, final Invoker invoker)
-            throws Exception {
+            throws Throwable {
         ActorRef actorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
 
         if (exToThrow instanceof PrimaryNotFoundException) {
@@ -158,23 +167,23 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         propagateReadFailedExceptionCause(invoker.invoke(transactionProxy));
     }
 
-    private void testReadWithExceptionOnInitialCreateTransaction(final Exception exToThrow) throws Exception {
+    private void testReadWithExceptionOnInitialCreateTransaction(final Exception exToThrow) throws Throwable {
         testExceptionOnInitialCreateTransaction(exToThrow, proxy -> proxy.read(TestModel.TEST_PATH));
     }
 
     @Test(expected = PrimaryNotFoundException.class)
-    public void testReadWhenAPrimaryNotFoundExceptionIsThrown() throws Exception {
+    public void testReadWhenAPrimaryNotFoundExceptionIsThrown() throws Throwable {
         testReadWithExceptionOnInitialCreateTransaction(new PrimaryNotFoundException("test"));
     }
 
-    @Test(expected = TimeoutException.class)
-    public void testReadWhenATimeoutExceptionIsThrown() throws Exception {
+    @Test(expected = TestException.class)
+    public void testReadWhenATimeoutExceptionIsThrown() throws Throwable {
         testReadWithExceptionOnInitialCreateTransaction(new TimeoutException("test",
-                new Exception("reason")));
+                new TestException()));
     }
 
     @Test(expected = TestException.class)
-    public void testReadWhenAnyOtherExceptionIsThrown() throws Exception {
+    public void testReadWhenAnyOtherExceptionIsThrown() throws Throwable {
         testReadWithExceptionOnInitialCreateTransaction(new TestException());
     }
 
@@ -196,7 +205,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
                 TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
 
-        assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
+        assertTrue("NormalizedNode isPresent", readOptional.isPresent());
         assertEquals("Response NormalizedNode", expectedNode, readOptional.get());
 
         InOrder inOrder = Mockito.inOrder(mockActorContext);
@@ -214,7 +223,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test(expected = IllegalArgumentException.class)
-    public void testInvalidCreateTransactionReply() throws Exception {
+    public void testInvalidCreateTransactionReply() throws Throwable {
         ActorRef actorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
 
         doReturn(getSystem().actorSelection(actorRef.path())).when(mockActorContext)
@@ -241,26 +250,26 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         doReturn(dataExistsReply(false)).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), eqDataExists(), any(Timeout.class));
 
-        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
+        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).get();
 
-        assertEquals("Exists response", false, exists);
+        assertEquals("Exists response", Boolean.FALSE, exists);
 
         doReturn(dataExistsReply(true)).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef)), eqDataExists(), any(Timeout.class));
 
-        exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
+        exists = transactionProxy.exists(TestModel.TEST_PATH).get();
 
-        assertEquals("Exists response", true, exists);
+        assertEquals("Exists response", Boolean.TRUE, exists);
     }
 
     @Test(expected = PrimaryNotFoundException.class)
-    public void testExistsWhenAPrimaryNotFoundExceptionIsThrown() throws Exception {
+    public void testExistsWhenAPrimaryNotFoundExceptionIsThrown() throws Throwable {
         testExceptionOnInitialCreateTransaction(new PrimaryNotFoundException("test"),
             proxy -> proxy.exists(TestModel.TEST_PATH));
     }
 
     @Test(expected = ReadFailedException.class)
-    public void testExistsWithInvalidReplyMessageType() throws Exception {
+    public void testExistsWithInvalidReplyMessageType() throws Throwable {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
 
         doReturn(Futures.successful(new Object())).when(mockActorContext)
@@ -268,11 +277,15 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         TransactionProxy transactionProxy = new TransactionProxy(mockComponentFactory, READ_ONLY);
 
-        transactionProxy.exists(TestModel.TEST_PATH).checkedGet(5, TimeUnit.SECONDS);
+        try {
+            transactionProxy.exists(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
+        } catch (ExecutionException e) {
+            throw e.getCause();
+        }
     }
 
     @Test(expected = TestException.class)
-    public void testExistsWithAsyncRemoteOperatonFailure() throws Exception {
+    public void testExistsWithAsyncRemoteOperatonFailure() throws Throwable {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_ONLY);
 
         doReturn(Futures.failed(new TestException())).when(mockActorContext)
@@ -298,9 +311,9 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         transactionProxy.write(TestModel.TEST_PATH, nodeToWrite);
 
-        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
+        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).get();
 
-        assertEquals("Exists response", true, exists);
+        assertEquals("Exists response", Boolean.TRUE, exists);
 
         InOrder inOrder = Mockito.inOrder(mockActorContext);
         inOrder.verify(mockActorContext).executeOperationAsync(
@@ -317,7 +330,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testWrite() throws Exception {
+    public void testWrite() {
         dataStoreContextBuilder.shardBatchedModificationCount(1);
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
@@ -406,7 +419,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testMerge() throws Exception {
+    public void testMerge() {
         dataStoreContextBuilder.shardBatchedModificationCount(1);
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
@@ -422,7 +435,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testDelete() throws Exception {
+    public void testDelete() {
         dataStoreContextBuilder.shardBatchedModificationCount(1);
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
@@ -436,7 +449,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadWrite() throws Exception {
+    public void testReadWrite() {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
         final NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
@@ -464,7 +477,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithReadWrite() throws Exception {
+    public void testReadyWithReadWrite() {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
         final NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
@@ -496,7 +509,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithNoModifications() throws Exception {
+    public void testReadyWithNoModifications() {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
         doReturn(readDataReply(null)).when(mockActorContext).executeOperationAsync(
@@ -521,29 +534,63 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithMultipleShardWrites() throws Exception {
+    public void testReadyWithMultipleShardWrites() {
         ActorRef actorRef1 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
-        ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY, "junk");
+        ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY,
+                TestModel.JUNK_QNAME.getLocalName());
 
         expectBatchedModificationsReady(actorRef1);
         expectBatchedModificationsReady(actorRef2);
 
+        ActorRef actorRef3 = getSystem().actorOf(Props.create(DoNothingActor.class));
+
+        doReturn(getSystem().actorSelection(actorRef3.path())).when(mockActorContext)
+                .actorSelection(actorRef3.path().toString());
+
+        doReturn(Futures.successful(newPrimaryShardInfo(actorRef3, createDataTree()))).when(mockActorContext)
+                .findPrimaryShardAsync(eq(CarsModel.BASE_QNAME.getLocalName()));
+
+        expectReadyLocalTransaction(actorRef3, false);
+
         TransactionProxy transactionProxy = new TransactionProxy(mockComponentFactory, WRITE_ONLY);
 
         transactionProxy.write(TestModel.JUNK_PATH, ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
         transactionProxy.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        transactionProxy.write(CarsModel.BASE_PATH, ImmutableNodes.containerNode(CarsModel.BASE_QNAME));
 
         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
 
         assertTrue(ready instanceof ThreePhaseCommitCohortProxy);
 
         verifyCohortFutures((ThreePhaseCommitCohortProxy)ready, actorSelection(actorRef1),
-                actorSelection(actorRef2));
+                actorSelection(actorRef2), actorSelection(actorRef3));
+
+        SortedSet<String> expShardNames =
+                ImmutableSortedSet.of(DefaultShardStrategy.DEFAULT_SHARD,
+                        TestModel.JUNK_QNAME.getLocalName(), CarsModel.BASE_QNAME.getLocalName());
+
+        ArgumentCaptor<BatchedModifications> batchedMods = ArgumentCaptor.forClass(BatchedModifications.class);
+        verify(mockActorContext).executeOperationAsync(
+                eq(actorSelection(actorRef1)), batchedMods.capture(), any(Timeout.class));
+        assertTrue("Participating shards present", batchedMods.getValue().getParticipatingShardNames().isPresent());
+        assertEquals("Participating shards", expShardNames, batchedMods.getValue().getParticipatingShardNames().get());
+
+        batchedMods = ArgumentCaptor.forClass(BatchedModifications.class);
+        verify(mockActorContext).executeOperationAsync(
+                eq(actorSelection(actorRef2)), batchedMods.capture(), any(Timeout.class));
+        assertTrue("Participating shards present", batchedMods.getValue().getParticipatingShardNames().isPresent());
+        assertEquals("Participating shards", expShardNames, batchedMods.getValue().getParticipatingShardNames().get());
+
+        ArgumentCaptor<ReadyLocalTransaction> readyLocalTx = ArgumentCaptor.forClass(ReadyLocalTransaction.class);
+        verify(mockActorContext).executeOperationAsync(
+                eq(actorSelection(actorRef3)), readyLocalTx.capture(), any(Timeout.class));
+        assertTrue("Participating shards present", readyLocalTx.getValue().getParticipatingShardNames().isPresent());
+        assertEquals("Participating shards", expShardNames, readyLocalTx.getValue().getParticipatingShardNames().get());
     }
 
     @Test
-    public void testReadyWithWriteOnlyAndLastBatchPending() throws Exception {
+    public void testReadyWithWriteOnlyAndLastBatchPending() {
         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
 
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
@@ -570,7 +617,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithWriteOnlyAndLastBatchEmpty() throws Exception {
+    public void testReadyWithWriteOnlyAndLastBatchEmpty() {
         dataStoreContextBuilder.shardBatchedModificationCount(1).writeOnlyTransactionOptimizationsEnabled(true);
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
@@ -598,7 +645,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithReplyFailure() throws Exception {
+    public void testReadyWithReplyFailure() {
         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
 
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
@@ -619,7 +666,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithDebugContextEnabled() throws Exception {
+    public void testReadyWithDebugContextEnabled() {
         dataStoreContextBuilder.transactionDebugContextEnabled(true);
 
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
@@ -638,7 +685,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadyWithLocalTransaction() throws Exception {
+    public void testReadyWithLocalTransaction() {
         ActorRef shardActorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
 
         doReturn(getSystem().actorSelection(shardActorRef.path())).when(mockActorContext)
@@ -657,10 +704,15 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         DOMStoreThreePhaseCommitCohort ready = transactionProxy.ready();
         assertTrue(ready instanceof SingleCommitCohortProxy);
         verifyCohortFutures((SingleCommitCohortProxy)ready, new CommitTransactionReply().toSerializable());
+
+        ArgumentCaptor<ReadyLocalTransaction> readyLocalTx = ArgumentCaptor.forClass(ReadyLocalTransaction.class);
+        verify(mockActorContext).executeOperationAsync(
+                eq(actorSelection(shardActorRef)), readyLocalTx.capture(), any(Timeout.class));
+        assertFalse("Participating shards present", readyLocalTx.getValue().getParticipatingShardNames().isPresent());
     }
 
     @Test
-    public void testReadyWithLocalTransactionWithFailure() throws Exception {
+    public void testReadyWithLocalTransactionWithFailure() {
         ActorRef shardActorRef = getSystem().actorOf(Props.create(DoNothingActor.class));
 
         doReturn(getSystem().actorSelection(shardActorRef.path())).when(mockActorContext)
@@ -685,7 +737,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         verifyCohortFutures((SingleCommitCohortProxy)ready, RuntimeException.class);
     }
 
-    private void testWriteOnlyTxWithFindPrimaryShardFailure(final Exception toThrow) throws Exception {
+    private void testWriteOnlyTxWithFindPrimaryShardFailure(final Exception toThrow) {
         doReturn(Futures.failed(toThrow)).when(mockActorContext).findPrimaryShardAsync(anyString());
 
         TransactionProxy transactionProxy = new TransactionProxy(mockComponentFactory, WRITE_ONLY);
@@ -706,26 +758,27 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testWriteOnlyTxWithPrimaryNotFoundException() throws Exception {
+    public void testWriteOnlyTxWithPrimaryNotFoundException() {
         testWriteOnlyTxWithFindPrimaryShardFailure(new PrimaryNotFoundException("mock"));
     }
 
     @Test
-    public void testWriteOnlyTxWithNotInitializedException() throws Exception {
+    public void testWriteOnlyTxWithNotInitializedException() {
         testWriteOnlyTxWithFindPrimaryShardFailure(new NotInitializedException("mock"));
     }
 
     @Test
-    public void testWriteOnlyTxWithNoShardLeaderException() throws Exception {
+    public void testWriteOnlyTxWithNoShardLeaderException() {
         testWriteOnlyTxWithFindPrimaryShardFailure(new NoShardLeaderException("mock"));
     }
 
     @Test
-    public void testReadyWithInvalidReplyMessageType() throws Exception {
+    public void testReadyWithInvalidReplyMessageType() {
         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
         ActorRef actorRef1 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY);
 
-        ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY, "junk");
+        ActorRef actorRef2 = setupActorContextWithInitialCreateTransaction(getSystem(), WRITE_ONLY,
+                TestModel.JUNK_QNAME.getLocalName());
 
         doReturn(Futures.successful(new Object())).when(mockActorContext).executeOperationAsync(
                 eq(actorSelection(actorRef1)), isA(BatchedModifications.class), any(Timeout.class));
@@ -756,7 +809,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testClose() throws Exception {
+    public void testClose() {
         ActorRef actorRef = setupActorContextWithInitialCreateTransaction(getSystem(), READ_WRITE);
 
         doReturn(readDataReply(null)).when(mockActorContext).executeOperationAsync(
@@ -1243,7 +1296,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
                 .getOperationTimeoutInMillis()) * 2);
     }
 
-    private void testModificationOperationBatching(final TransactionType type) throws Exception {
+    private void testModificationOperationBatching(final TransactionType type) {
         int shardBatchedModificationCount = 3;
         dataStoreContextBuilder.shardBatchedModificationCount(shardBatchedModificationCount);
 
@@ -1302,17 +1355,17 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadWriteModificationOperationBatching() throws Exception {
+    public void testReadWriteModificationOperationBatching() {
         testModificationOperationBatching(READ_WRITE);
     }
 
     @Test
-    public void testWriteOnlyModificationOperationBatching() throws Exception {
+    public void testWriteOnlyModificationOperationBatching() {
         testModificationOperationBatching(WRITE_ONLY);
     }
 
     @Test
-    public void testOptimizedWriteOnlyModificationOperationBatching() throws Exception {
+    public void testOptimizedWriteOnlyModificationOperationBatching() {
         dataStoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
         testModificationOperationBatching(WRITE_ONLY);
     }
@@ -1357,7 +1410,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(writePath2).get(5, TimeUnit.SECONDS);
 
-        assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
+        assertTrue("NormalizedNode isPresent", readOptional.isPresent());
         assertEquals("Response NormalizedNode", writeNode2, readOptional.get());
 
         transactionProxy.merge(mergePath1, mergeNode1);
@@ -1367,10 +1420,10 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
 
         transactionProxy.delete(deletePath);
 
-        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).checkedGet();
-        assertEquals("Exists response", true, exists);
+        Boolean exists = transactionProxy.exists(TestModel.TEST_PATH).get();
+        assertEquals("Exists response", Boolean.TRUE, exists);
 
-        assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
+        assertTrue("NormalizedNode isPresent", readOptional.isPresent());
         assertEquals("Response NormalizedNode", mergeNode2, readOptional.get());
 
         List<BatchedModifications> batchedModifications = captureBatchedModifications(actorRef);
@@ -1405,7 +1458,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
     }
 
     @Test
-    public void testReadRoot() throws ReadFailedException, InterruptedException, ExecutionException,
+    public void testReadRoot() throws InterruptedException, ExecutionException,
             java.util.concurrent.TimeoutException {
         SchemaContext schemaContext = SchemaContextHelper.full();
         Configuration configuration = mock(Configuration.class);
@@ -1428,7 +1481,7 @@ public class TransactionProxyTest extends AbstractTransactionProxyTest {
         Optional<NormalizedNode<?, ?>> readOptional = transactionProxy.read(
                 YangInstanceIdentifier.EMPTY).get(5, TimeUnit.SECONDS);
 
-        assertEquals("NormalizedNode isPresent", true, readOptional.isPresent());
+        assertTrue("NormalizedNode isPresent", readOptional.isPresent());
 
         NormalizedNode<?, ?> normalizedNode = readOptional.get();