This is an automated email from the ASF dual-hosted git repository.

mapohl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 1ae8e401b0a [FLINK-32857][JUnit5 Migration] Migrate the webmonitor and 
zookeeper packages of flink-runtime module to junit5 (#23196)
1ae8e401b0a is described below

commit 1ae8e401b0ad7c185b1d166d6fcfd879a877bf9f
Author: Rui Fan <[email protected]>
AuthorDate: Wed Aug 23 21:43:09 2023 +0800

    [FLINK-32857][JUnit5 Migration] Migrate the webmonitor and zookeeper 
packages of flink-runtime module to junit5 (#23196)
---
 .../flink/runtime/rpc/TestingRpcGateway.java       |  70 +++
 .../runtime/webmonitor/WebMonitorEndpointTest.java |   7 +-
 .../webmonitor/history/ArchivedJsonTest.java       |  25 +-
 .../webmonitor/history/HistoryServerUtilsTest.java |  36 +-
 .../retriever/LeaderGatewayRetrieverTest.java      |  31 +-
 .../retriever/impl/RpcGatewayRetrieverTest.java    |  47 +-
 .../ThreadInfoRequestCoordinatorTest.java          |   4 +-
 .../threadinfo/VertexThreadInfoTrackerTest.java    |  23 +-
 .../zookeeper/ZooKeeperStateHandleStoreTest.java   | 686 +++++++++++----------
 9 files changed, 494 insertions(+), 435 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcGateway.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcGateway.java
new file mode 100644
index 00000000000..f248dcc53da
--- /dev/null
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcGateway.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc;
+
+import java.util.function.Supplier;
+
+/** {@code TestingRpcGateway} is a generic test implementation of {@link 
RpcGateway}. */
+public class TestingRpcGateway implements RpcGateway {
+
+    private final Supplier<String> addressSupplier;
+    private final Supplier<String> hostnameSupplier;
+
+    private TestingRpcGateway(Supplier<String> addressSupplier, 
Supplier<String> hostnameSupplier) {
+        this.addressSupplier = addressSupplier;
+        this.hostnameSupplier = hostnameSupplier;
+    }
+
+    @Override
+    public String getAddress() {
+        return addressSupplier.get();
+    }
+
+    @Override
+    public String getHostname() {
+        return hostnameSupplier.get();
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /** {@code Builder} for {@code TestingRpcGateway}. */
+    public static class Builder {
+
+        private Supplier<String> addressSupplier = () -> "address";
+        private Supplier<String> hostnameSupplier = () -> "hostname";
+
+        private Builder() {}
+
+        public Builder setAddressSupplier(Supplier<String> addressSupplier) {
+            this.addressSupplier = addressSupplier;
+            return this;
+        }
+
+        public Builder setHostnameSupplier(Supplier<String> hostnameSupplier) {
+            this.hostnameSupplier = hostnameSupplier;
+            return this;
+        }
+
+        public TestingRpcGateway build() {
+            return new TestingRpcGateway(addressSupplier, hostnameSupplier);
+        }
+    }
+}
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpointTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpointTest.java
index e7c59d5d5dd..ffb6966e661 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpointTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpointTest.java
@@ -28,9 +28,8 @@ import 
org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
 import org.apache.flink.runtime.rest.handler.legacy.metrics.VoidMetricFetcher;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.apache.flink.util.ExecutorUtils;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
@@ -39,10 +38,10 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
 /** Tests for the {@link WebMonitorEndpoint}. */
-public class WebMonitorEndpointTest extends TestLogger {
+class WebMonitorEndpointTest {
 
     @Test
-    public void cleansUpExpiredExecutionGraphs() throws Exception {
+    void cleansUpExpiredExecutionGraphs() throws Exception {
         final Configuration configuration = new Configuration();
         configuration.setString(RestOptions.ADDRESS, "localhost");
         configuration.setLong(WebOptions.REFRESH_INTERVAL, 5L);
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/ArchivedJsonTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/ArchivedJsonTest.java
index c0a00ad3501..074a2599452 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/ArchivedJsonTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/ArchivedJsonTest.java
@@ -18,32 +18,33 @@
 
 package org.apache.flink.runtime.webmonitor.history;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for the {@link ArchivedJson}. */
-public class ArchivedJsonTest {
+class ArchivedJsonTest {
 
     @Test
-    public void testEquals() {
+    void testEquals() {
         ArchivedJson original = new ArchivedJson("path", "json");
         ArchivedJson twin = new ArchivedJson("path", "json");
         ArchivedJson identicalPath = new ArchivedJson("path", "hello");
         ArchivedJson identicalJson = new ArchivedJson("hello", "json");
 
-        Assert.assertEquals(original, original);
-        Assert.assertEquals(original, twin);
-        Assert.assertNotEquals(original, identicalPath);
-        Assert.assertNotEquals(original, identicalJson);
+        assertThat(original).isEqualTo(original);
+        assertThat(twin).isEqualTo(original);
+        assertThat(identicalPath).isNotEqualTo(original);
+        assertThat(identicalJson).isNotEqualTo(original);
     }
 
     @Test
-    public void testHashCode() {
+    void testHashCode() {
         ArchivedJson original = new ArchivedJson("path", "json");
         ArchivedJson twin = new ArchivedJson("path", "json");
 
-        Assert.assertEquals(original, original);
-        Assert.assertEquals(original, twin);
-        Assert.assertEquals(original.hashCode(), twin.hashCode());
+        assertThat(original).isEqualTo(original);
+        assertThat(twin).isEqualTo(original);
+        assertThat(twin).hasSameHashCodeAs(original);
     }
 }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerUtilsTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerUtilsTest.java
index f28bea732df..56c48d931e9 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerUtilsTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerUtilsTest.java
@@ -21,9 +21,8 @@ package org.apache.flink.runtime.webmonitor.history;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HistoryServerOptions;
 import org.apache.flink.configuration.SecurityOptions;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nonnull;
 
@@ -31,44 +30,43 @@ import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.Optional;
 
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for the {@link HistoryServerUtils}. */
-public class HistoryServerUtilsTest extends TestLogger {
+class HistoryServerUtilsTest {
 
     private static final String HOSTNAME = "foobar";
     private static final int PORT = 1234;
 
     @Test
-    public void testIsSSLEnabledDefault() {
+    void testIsSSLEnabledDefault() {
         final Configuration configuration = new Configuration();
 
-        assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
+        assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
     }
 
     @Test
-    public void testIsSSLEnabledWithoutRestSSL() {
+    void testIsSSLEnabledWithoutRestSSL() {
         final Configuration configuration = new Configuration();
         
configuration.setBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED, 
true);
 
-        assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
+        assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
     }
 
     @Test
-    public void testIsSSLEnabledOnlyRestSSL() {
+    void testIsSSLEnabledOnlyRestSSL() {
         final Configuration configuration = new Configuration();
         configuration.setBoolean(SecurityOptions.SSL_REST_ENABLED, true);
 
-        assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
+        assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
     }
 
     @Test
-    public void testIsSSLEnabled() {
+    void testIsSSLEnabled() {
         final Configuration configuration = new Configuration();
         enableSSL(configuration);
 
-        assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(true));
+        assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isTrue();
     }
 
     private void enableSSL(Configuration configuration) {
@@ -77,31 +75,31 @@ public class HistoryServerUtilsTest extends TestLogger {
     }
 
     @Test
-    public void testGetHistoryServerURL() throws MalformedURLException {
+    void testGetHistoryServerURL() throws MalformedURLException {
         final Configuration configuration = createDefaultConfiguration();
 
         final Optional<URL> historyServerURL =
                 HistoryServerUtils.getHistoryServerURL(configuration);
 
-        assertThat(historyServerURL.get(), is(new URL("http", HOSTNAME, PORT, 
"")));
+        assertThat(historyServerURL).isPresent().hasValue(new URL("http", 
HOSTNAME, PORT, ""));
     }
 
     @Test
-    public void testGetHistoryServerURLWithSSL() throws MalformedURLException {
+    void testGetHistoryServerURLWithSSL() throws MalformedURLException {
         final Configuration configuration = createDefaultConfiguration();
         enableSSL(configuration);
 
         final Optional<URL> historyServerURL =
                 HistoryServerUtils.getHistoryServerURL(configuration);
 
-        assertThat(historyServerURL.get(), is(new URL("https", HOSTNAME, PORT, 
"")));
+        assertThat(historyServerURL).isPresent().hasValue(new URL("https", 
HOSTNAME, PORT, ""));
     }
 
     @Test
-    public void testGetHistoryServerURLWithoutHS() {
+    void testGetHistoryServerURLWithoutHS() {
         final Configuration configuration = new Configuration();
 
-        
assertThat(HistoryServerUtils.getHistoryServerURL(configuration).isPresent(), 
is(false));
+        
assertThat(HistoryServerUtils.getHistoryServerURL(configuration)).isNotPresent();
     }
 
     @Nonnull
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/LeaderGatewayRetrieverTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/LeaderGatewayRetrieverTest.java
index 6ac218191e1..0972cd00f56 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/LeaderGatewayRetrieverTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/LeaderGatewayRetrieverTest.java
@@ -19,33 +19,31 @@
 package org.apache.flink.runtime.webmonitor.retriever;
 
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.testutils.FlinkAssertions;
 import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
 import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.TestingRpcGateway;
 import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.TestLogger;
 import org.apache.flink.util.concurrent.FutureUtils;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Test cases for the {@link LeaderGatewayRetriever}. */
-public class LeaderGatewayRetrieverTest extends TestLogger {
+class LeaderGatewayRetrieverTest {
 
     /** Tests that the gateway retrieval is retried in case of a failure. */
     @Test
-    public void testGatewayRetrievalFailures() throws Exception {
+    void testGatewayRetrievalFailures() throws Exception {
         final String address = "localhost";
         final UUID leaderId = UUID.randomUUID();
 
-        RpcGateway rpcGateway = mock(RpcGateway.class);
+        RpcGateway rpcGateway = TestingRpcGateway.newBuilder().build();
 
         TestingLeaderGatewayRetriever leaderGatewayRetriever =
                 new TestingLeaderGatewayRetriever(rpcGateway);
@@ -59,20 +57,15 @@ public class LeaderGatewayRetrieverTest extends TestLogger {
         // this triggers the first gateway retrieval attempt
         settableLeaderRetrievalService.notifyListener(address, leaderId);
 
-        // check that the first future has been failed
-        try {
-            gatewayFuture.get();
-
-            fail("The first future should have been failed.");
-        } catch (ExecutionException ignored) {
-            // that's what we expect
-        }
+        FlinkAssertions.assertThatFuture(gatewayFuture)
+                .as("The first future should have been failed.")
+                .eventuallyFailsWith(ExecutionException.class);
 
         // the second attempt should fail as well
-        assertFalse((leaderGatewayRetriever.getNow().isPresent()));
+        assertThat((leaderGatewayRetriever.getNow())).isNotPresent();
 
         // the third attempt should succeed
-        assertEquals(rpcGateway, leaderGatewayRetriever.getNow().get());
+        assertThat(leaderGatewayRetriever.getNow()).hasValue(rpcGateway);
     }
 
     private static class TestingLeaderGatewayRetriever extends 
LeaderGatewayRetriever<RpcGateway> {
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java
index 2d6d81773d2..65801e2031d 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/retriever/impl/RpcGatewayRetrieverTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.webmonitor.retriever.impl;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.core.testutils.FlinkAssertions;
 import org.apache.flink.runtime.dispatcher.cleanup.TestingRetryStrategies;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
@@ -28,36 +29,32 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.RpcUtils;
 import org.apache.flink.runtime.rpc.TestingRpcService;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.function.Function;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for the {@link RpcGatewayRetriever}. */
-public class RpcGatewayRetrieverTest extends TestLogger {
+class RpcGatewayRetrieverTest {
 
     private static final Time TIMEOUT = Time.seconds(10L);
     private static TestingRpcService rpcService;
 
-    @BeforeClass
-    public static void setup() {
+    @BeforeAll
+    static void setup() {
         rpcService = new TestingRpcService();
     }
 
-    @AfterClass
-    public static void teardown()
-            throws InterruptedException, ExecutionException, TimeoutException {
+    @AfterAll
+    static void teardown() throws InterruptedException, ExecutionException {
         if (rpcService != null) {
             RpcUtils.terminateRpcService(rpcService);
             rpcService = null;
@@ -69,7 +66,7 @@ public class RpcGatewayRetrieverTest extends TestLogger {
      * retrieval service.
      */
     @Test
-    public void testRpcGatewayRetrieval() throws Exception {
+    void testRpcGatewayRetrieval() throws Exception {
         final String expectedValue = "foobar";
         final String expectedValue2 = "barfoo";
         final UUID leaderSessionId = UUID.randomUUID();
@@ -100,7 +97,7 @@ public class RpcGatewayRetrieverTest extends TestLogger {
 
             final CompletableFuture<DummyGateway> gatewayFuture = 
gatewayRetriever.getFuture();
 
-            assertFalse(gatewayFuture.isDone());
+            assertThat(gatewayFuture).isNotDone();
 
             settableLeaderRetrievalService.notifyListener(
                     dummyRpcEndpoint.getAddress(), leaderSessionId);
@@ -108,12 +105,10 @@ public class RpcGatewayRetrieverTest extends TestLogger {
             final DummyGateway dummyGateway =
                     gatewayFuture.get(TIMEOUT.toMilliseconds(), 
TimeUnit.MILLISECONDS);
 
-            assertEquals(dummyRpcEndpoint.getAddress(), 
dummyGateway.getAddress());
-            assertEquals(
-                    expectedValue,
-                    dummyGateway
-                            .foobar(TIMEOUT)
-                            .get(TIMEOUT.toMilliseconds(), 
TimeUnit.MILLISECONDS));
+            
assertThat(dummyGateway.getAddress()).isEqualTo(dummyRpcEndpoint.getAddress());
+            FlinkAssertions.assertThatFuture(dummyGateway.foobar(TIMEOUT))
+                    .eventuallySucceeds()
+                    .isEqualTo(expectedValue);
 
             // elect a new leader
             settableLeaderRetrievalService.notifyListener(
@@ -123,12 +118,10 @@ public class RpcGatewayRetrieverTest extends TestLogger {
             final DummyGateway dummyGateway2 =
                     gatewayFuture2.get(TIMEOUT.toMilliseconds(), 
TimeUnit.MILLISECONDS);
 
-            assertEquals(dummyRpcEndpoint2.getAddress(), 
dummyGateway2.getAddress());
-            assertEquals(
-                    expectedValue2,
-                    dummyGateway2
-                            .foobar(TIMEOUT)
-                            .get(TIMEOUT.toMilliseconds(), 
TimeUnit.MILLISECONDS));
+            
assertThat(dummyGateway2.getAddress()).isEqualTo(dummyRpcEndpoint2.getAddress());
+            FlinkAssertions.assertThatFuture(dummyGateway2.foobar(TIMEOUT))
+                    .eventuallySucceeds()
+                    .isEqualTo(expectedValue2);
         } finally {
             RpcUtils.terminateRpcEndpoint(dummyRpcEndpoint, dummyRpcEndpoint2);
         }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/ThreadInfoRequestCoordinatorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/ThreadInfoRequestCoordinatorTest.java
index d38c6c745d7..4dfcade0777 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/ThreadInfoRequestCoordinatorTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/ThreadInfoRequestCoordinatorTest.java
@@ -70,12 +70,12 @@ class ThreadInfoRequestCoordinatorTest {
     private ThreadInfoRequestCoordinator coordinator;
 
     @BeforeAll
-    static void setUp() throws Exception {
+    static void setUp() {
         executorService = new ScheduledThreadPoolExecutor(1);
     }
 
     @AfterAll
-    static void tearDown() throws Exception {
+    static void tearDown() {
         if (executorService != null) {
             executorService.shutdown();
         }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexThreadInfoTrackerTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexThreadInfoTrackerTest.java
index bbf5f7a1392..86462ad5b5e 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexThreadInfoTrackerTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/threadinfo/VertexThreadInfoTrackerTest.java
@@ -37,7 +37,6 @@ import 
org.apache.flink.runtime.taskexecutor.TaskExecutorThreadInfoGateway;
 import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
 import org.apache.flink.runtime.util.JvmUtils;
 import org.apache.flink.testutils.TestingUtils;
-import org.apache.flink.util.TestLogger;
 
 import org.apache.flink.shaded.guava31.com.google.common.cache.Cache;
 import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder;
@@ -79,7 +78,7 @@ import static org.apache.flink.util.Preconditions.checkState;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
 
 /** Tests for the {@link VertexThreadInfoTracker}. */
-public class VertexThreadInfoTrackerTest extends TestLogger {
+class VertexThreadInfoTrackerTest {
 
     private static final int REQUEST_ID = 0;
     private static final int PARALLELISM = 10;
@@ -108,7 +107,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
     private static ScheduledExecutorService executor;
 
     @BeforeAll
-    public static void setUp() {
+    static void setUp() {
         // Time gap determines endTime of stats, which controls if the 
"refresh" is triggered:
         // now >= stats.getEndTime() + statsRefreshInterval
         // Using a small gap to be able to test cache updates without much 
delay.
@@ -117,7 +116,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
     }
 
     @AfterAll
-    public static void tearDown() {
+    static void tearDown() {
         if (executor != null) {
             executor.shutdownNow();
         }
@@ -125,7 +124,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests successful thread info stats request. */
     @Test
-    public void testGetThreadInfoStats() throws Exception {
+    void testGetThreadInfoStats() throws Exception {
         doInitialJobVertexRequestAndVerifyResult(createThreadInfoTracker());
         for (int subtaskIndex = 0; subtaskIndex < PARALLELISM; subtaskIndex++) 
{
             
doInitialExecutionVertexRequestAndVerifyResult(createThreadInfoTracker(), 
subtaskIndex);
@@ -134,7 +133,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached result is reused within refresh interval. */
     @Test
-    public void testCachedStatsNotUpdatedWithinRefreshInterval() throws 
Exception {
+    void testCachedStatsNotUpdatedWithinRefreshInterval() throws Exception {
         final VertexThreadInfoStats unusedThreadInfoStats = 
createThreadInfoStats(1, TIME_GAP);
 
         // Test for trigger request at job vertex level
@@ -175,7 +174,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached job vertex result is NOT reused after refresh 
interval. */
     @Test
-    public void testJobVertexCachedStatsUpdatedAfterRefreshInterval() throws 
Exception {
+    void testJobVertexCachedStatsUpdatedAfterRefreshInterval() throws 
Exception {
         final Duration shortRefreshInterval = Duration.ofMillis(1000);
 
         // first entry is in the past, so refresh is triggered immediately 
upon fetching it
@@ -220,7 +219,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached execution vertex result is NOT reused after refresh 
interval. */
     @Test
-    public void testExecutionVertexCachedStatsUpdatedAfterRefreshInterval() 
throws Exception {
+    void testExecutionVertexCachedStatsUpdatedAfterRefreshInterval() throws 
Exception {
         final Duration shortRefreshInterval = Duration.ofMillis(1000);
 
         // first entry is in the past, so refresh is triggered immediately 
upon fetching it
@@ -278,7 +277,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
      * request is pending.
      */
     @Test
-    public void testExecutionVertexShouldBeIgnoredWhenJobVertexIsPending() 
throws Exception {
+    void testExecutionVertexShouldBeIgnoredWhenJobVertexIsPending() throws 
Exception {
         CompletableFuture<VertexThreadInfoStats> statsFuture = new 
CompletableFuture<>();
         TestingBlockingAndCountableCoordinator coordinator =
                 new TestingBlockingAndCountableCoordinator(statsFuture);
@@ -314,7 +313,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached results are removed within the cleanup interval. */
     @Test
-    public void testCachedStatsCleanedAfterCleanupInterval() throws Exception {
+    void testCachedStatsCleanedAfterCleanupInterval() throws Exception {
         final Duration shortCleanUpInterval = Duration.ofMillis(1);
 
         // register a CountDownLatch with the cache so we can await expiry of 
the entry
@@ -371,7 +370,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached results are NOT removed within the cleanup interval. 
*/
     @Test
-    public void testCachedStatsNotCleanedWithinCleanupInterval() throws 
Exception {
+    void testCachedStatsNotCleanedWithinCleanupInterval() throws Exception {
         final VertexThreadInfoTracker tracker = createThreadInfoTracker();
 
         doInitialJobVertexRequestAndVerifyResult(tracker);
@@ -391,7 +390,7 @@ public class VertexThreadInfoTrackerTest extends TestLogger 
{
 
     /** Tests that cached results are not served after the shutdown. */
     @Test
-    public void testShutDown() throws Exception {
+    void testShutDown() throws Exception {
         final VertexThreadInfoTracker tracker = createThreadInfoTracker();
         doInitialJobVertexRequestAndVerifyResult(tracker);
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java
index 7f67d65e1f2..df5fe89d6cf 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.zookeeper;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
 import 
org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener;
 import org.apache.flink.runtime.persistence.IntegerResourceVersion;
 import org.apache.flink.runtime.persistence.PossibleInconsistentStateException;
@@ -29,43 +30,31 @@ import 
org.apache.flink.runtime.persistence.StateHandleStore;
 import org.apache.flink.runtime.persistence.TestingLongStateHandleHelper;
 import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler;
 import org.apache.flink.runtime.state.RetrievableStateHandle;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerExtension;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
-import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.TestLogger;
 
 import 
org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;
 import org.apache.flink.shaded.guava31.com.google.common.collect.Iterables;
 import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
 import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
 
-import org.hamcrest.collection.IsIterableContainingInAnyOrder;
-import org.hamcrest.core.IsInstanceOf;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 import static 
org.apache.flink.runtime.util.ZooKeeperUtils.generateZookeeperPath;
-import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertThrows;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
@@ -79,27 +68,34 @@ import static org.mockito.Mockito.when;
  *   <li>Correct ordering of ZooKeeper and state handle operations
  * </ul>
  */
-public class ZooKeeperStateHandleStoreTest extends TestLogger {
+class ZooKeeperStateHandleStoreTest {
 
-    private static final ZooKeeperTestEnvironment ZOOKEEPER = new 
ZooKeeperTestEnvironment(1);
+    private final ZooKeeperExtension zooKeeperExtension = new 
ZooKeeperExtension();
 
-    @AfterClass
-    public static void tearDown() throws Exception {
-        ZOOKEEPER.shutdown();
-    }
+    @RegisterExtension
+    final EachCallbackWrapper<ZooKeeperExtension> zooKeeperResource =
+            new EachCallbackWrapper<>(zooKeeperExtension);
+
+    @RegisterExtension
+    final TestingFatalErrorHandlerExtension testingFatalErrorHandlerResource =
+            new TestingFatalErrorHandlerExtension();
 
-    @Before
-    public void cleanUp() throws Exception {
-        ZOOKEEPER.deleteAll();
+    @BeforeEach
+    void cleanUp() {
         TestingLongStateHandleHelper.clearGlobalState();
     }
 
+    private CuratorFramework getZooKeeperClient() {
+        return zooKeeperExtension.getZooKeeperClient(
+                
testingFatalErrorHandlerResource.getTestingFatalErrorHandler());
+    }
+
     /** Tests add operation with lock. */
     @Test
-    public void testAddAndLock() throws Exception {
+    void testAddAndLock() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         // Config
         final String pathInZooKeeper = "/testAdd";
@@ -110,62 +106,61 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         // Verify
         // State handle created
-        assertEquals(1, store.getAllAndLock().size());
-        assertEquals(state, 
store.getAndLock(pathInZooKeeper).retrieveState().getValue());
+        assertThat(store.getAllAndLock()).hasSize(1);
+        
assertThat(store.getAndLock(pathInZooKeeper).retrieveState().getValue()).isEqualTo(state);
 
         // Path created and is persistent
-        Stat stat = 
ZOOKEEPER.getClient().checkExists().forPath(pathInZooKeeper);
-        assertNotNull(stat);
-        assertEquals(0, stat.getEphemeralOwner());
+        Stat stat = 
getZooKeeperClient().checkExists().forPath(pathInZooKeeper);
+        assertThat(stat).isNotNull();
+        assertThat(stat.getEphemeralOwner()).isZero();
 
-        List<String> children = 
ZOOKEEPER.getClient().getChildren().forPath(pathInZooKeeper);
+        List<String> children = 
getZooKeeperClient().getChildren().forPath(pathInZooKeeper);
 
         // There should be one child which is the locks subfolder
         final String locksSubfolderChild = Iterables.getOnlyElement(children);
         stat =
-                ZOOKEEPER
-                        .getClient()
+                getZooKeeperClient()
                         .checkExists()
                         .forPath(generateZookeeperPath(pathInZooKeeper, 
locksSubfolderChild));
-        assertNotNull(stat);
+        assertThat(stat).isNotNull();
 
-        assertEquals("The lock subfolder shouldn't be ephemeral", 0, 
stat.getEphemeralOwner());
+        assertThat(stat.getEphemeralOwner())
+                .as("The lock subfolder shouldn't be ephemeral")
+                .isZero();
 
         List<String> lockChildren =
-                ZOOKEEPER
-                        .getClient()
+                getZooKeeperClient()
                         .getChildren()
                         .forPath(generateZookeeperPath(pathInZooKeeper, 
locksSubfolderChild));
         // Only one lock is expected
         final String lockChild = Iterables.getOnlyElement(lockChildren);
         stat =
-                ZOOKEEPER
-                        .getClient()
+                getZooKeeperClient()
                         .checkExists()
                         .forPath(
                                 generateZookeeperPath(
                                         pathInZooKeeper, locksSubfolderChild, 
lockChild));
 
-        assertNotEquals("The lock node should be ephemeral", 0, 
stat.getEphemeralOwner());
+        assertThat(stat.getEphemeralOwner()).as("The lock node should be 
ephemeral").isNotZero();
 
         // Data is equal
         @SuppressWarnings("unchecked")
         final long actual =
                 
((RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>)
                                 InstantiationUtil.deserializeObject(
-                                        
ZOOKEEPER.getClient().getData().forPath(pathInZooKeeper),
+                                        
getZooKeeperClient().getData().forPath(pathInZooKeeper),
                                         ClassLoader.getSystemClassLoader()))
                         .retrieveState()
                         .getValue();
 
-        assertEquals(state, actual);
+        assertThat(actual).isEqualTo(state);
     }
 
     @Test
-    public void testAddAndLockOnMarkedForDeletionNode() throws Exception {
+    void testAddAndLockOnMarkedForDeletionNode() throws Exception {
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testAddAndLockOnMarkedForDeletionNode");
+                        getZooKeeperClient(), 
"/testAddAndLockOnMarkedForDeletionNode");
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
                 new ZooKeeperStateHandleStore<>(client, new 
TestingLongStateHandleHelper());
 
@@ -183,18 +178,17 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
                 new 
TestingLongStateHandleHelper.LongStateHandle(updatedStateValue);
         zkStore.addAndLock(markedForDeletionNodePath, updatedStateHandle);
 
-        assertEquals(
-                updatedStateValue,
-                
zkStore.getAndLock(markedForDeletionNodePath).retrieveState().getValue());
-        assertTrue(oldStateHandle.isDiscarded());
-        assertFalse(updatedStateHandle.isDiscarded());
+        
assertThat(zkStore.getAndLock(markedForDeletionNodePath).retrieveState().getValue())
+                .isEqualTo(updatedStateValue);
+        assertThat(oldStateHandle.isDiscarded()).isTrue();
+        assertThat(updatedStateHandle.isDiscarded()).isFalse();
     }
 
     @Test
-    public void testRepeatableCleanup() throws Exception {
+    void testRepeatableCleanup() throws Exception {
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
testInstance =
                 new ZooKeeperStateHandleStore<>(
-                        ZOOKEEPER.getClient(), new 
TestingLongStateHandleHelper());
+                        getZooKeeperClient(), new 
TestingLongStateHandleHelper());
 
         final String pathInZooKeeper = "/testRepeatableCleanup";
 
@@ -206,40 +200,36 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         testInstance.addAndLock(pathInZooKeeper, stateHandle);
 
-        try {
-            testInstance.releaseAndTryRemove(pathInZooKeeper);
-            fail("Exception should have been thrown.");
-        } catch (Exception e) {
-            ExceptionUtils.assertThrowable(e, expectedException::equals);
-        }
+        assertThatThrownBy(() -> 
testInstance.releaseAndTryRemove(pathInZooKeeper))
+                .as("Exception should have been thrown.")
+                .hasCause(expectedException);
 
-        assertThrows(
-                StateHandleStore.NotExistException.class,
-                () -> testInstance.getAndLock(pathInZooKeeper));
-        assertFalse(stateHandle.isDiscarded());
+        assertThatExceptionOfType(StateHandleStore.NotExistException.class)
+                .isThrownBy(() -> testInstance.getAndLock(pathInZooKeeper));
+        assertThat(stateHandle.isDiscarded()).isFalse();
 
-        assertTrue(testInstance.releaseAndTryRemove(pathInZooKeeper));
-        assertFalse(testInstance.exists(pathInZooKeeper).isExisting());
-        assertTrue(stateHandle.isDiscarded());
+        assertThat(testInstance.releaseAndTryRemove(pathInZooKeeper)).isTrue();
+        
assertThat(testInstance.exists(pathInZooKeeper).isExisting()).isFalse();
+        assertThat(stateHandle.isDiscarded()).isTrue();
     }
 
     @Test
-    public void testCleanupOfNonExistingState() throws Exception {
+    void testCleanupOfNonExistingState() throws Exception {
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
testInstance =
                 new ZooKeeperStateHandleStore<>(
-                        ZOOKEEPER.getClient(), new 
TestingLongStateHandleHelper());
+                        getZooKeeperClient(), new 
TestingLongStateHandleHelper());
 
         final String pathInZooKeeper = "/testCleanupOfNonExistingState";
 
-        assertTrue(testInstance.releaseAndTryRemove(pathInZooKeeper));
-        assertFalse(testInstance.exists(pathInZooKeeper).isExisting());
+        assertThat(testInstance.releaseAndTryRemove(pathInZooKeeper)).isTrue();
+        
assertThat(testInstance.exists(pathInZooKeeper).isExisting()).isFalse();
     }
 
     @Test
-    public void testRepeatableCleanupWithLockOnNode() throws Exception {
+    void testRepeatableCleanupWithLockOnNode() throws Exception {
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testRepeatableCleanupWithLockOnNode");
+                        getZooKeeperClient(), 
"/testRepeatableCleanupWithLockOnNode");
 
         final TestingLongStateHandleHelper storage = new 
TestingLongStateHandleHelper();
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle>
@@ -262,47 +252,53 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         final 
RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>
                 handleForDeletion = 
storeForDeletion.getAndLock(pathInZooKeeper);
 
-        assertEquals(handleForCreation.retrieveState().getValue(), 
actualValue);
-        assertEquals(handleForDeletion.retrieveState().getValue(), 
actualValue);
-
-        assertFalse(
-                "Deletion by the first StateHandleStore shouldn't be 
successful because there's still a lock from StateHandleStore #1.",
-                storeForCreation.releaseAndTryRemove(pathInZooKeeper));
-
-        assertNotNull(
-                "StateHandle should not be marked for deletion, yet.",
-                client.checkExists()
-                        
.forPath(ZooKeeperStateHandleStore.getRootLockPath(pathInZooKeeper)));
-        assertNull(
-                "The lock for storeForCreation should have been removed",
-                client.checkExists()
-                        
.forPath(storeForCreation.getInstanceLockPath(pathInZooKeeper)));
-        assertEquals(
-                "discardState shouldn't have been called, yet.",
-                0,
-                stateHandle.getNumberOfDiscardCalls());
-
-        try {
-            storeForDeletion.releaseAndTryRemove(pathInZooKeeper);
-            fail("Exception should have been thrown.");
-        } catch (Exception e) {
-            ExceptionUtils.assertThrowable(e, expectedException::equals);
-        }
+        
assertThat(actualValue).isEqualTo(handleForCreation.retrieveState().getValue());
+        
assertThat(actualValue).isEqualTo(handleForDeletion.retrieveState().getValue());
+
+        assertThat(storeForCreation.releaseAndTryRemove(pathInZooKeeper))
+                .as(
+                        "Deletion by the first StateHandleStore shouldn't be 
successful because there's still a lock from StateHandleStore #1.")
+                .isFalse();
 
-        assertNull(
-                "StateHandle should be marked for deletion.",
-                client.checkExists()
-                        
.forPath(ZooKeeperStateHandleStore.getRootLockPath(pathInZooKeeper)));
-        assertNotNull(
-                "StateHandle should not be deleted, yet.",
-                client.checkExists().forPath(pathInZooKeeper));
-        assertFalse("The StateHandle should not be discarded, yet.", 
stateHandle.isDiscarded());
-
-        assertTrue(storeForDeletion.releaseAndTryRemove(pathInZooKeeper));
-        assertNull(
-                "The StateHandle node should have been removed",
-                client.checkExists().forPath(pathInZooKeeper));
-        assertTrue("The StateHandle should have been discarded.", 
stateHandle.isDiscarded());
+        assertThat(
+                        client.checkExists()
+                                .forPath(
+                                        
ZooKeeperStateHandleStore.getRootLockPath(pathInZooKeeper)))
+                .as("StateHandle should not be marked for deletion, yet.")
+                .isNotNull();
+        assertThat(
+                        client.checkExists()
+                                
.forPath(storeForCreation.getInstanceLockPath(pathInZooKeeper)))
+                .as("The lock for storeForCreation should have been removed")
+                .isNull();
+        assertThat(stateHandle.getNumberOfDiscardCalls())
+                .as("discardState shouldn't have been called, yet.")
+                .isZero();
+
+        assertThatThrownBy(() -> 
storeForDeletion.releaseAndTryRemove(pathInZooKeeper))
+                .as("Exception should have been thrown.")
+                .hasCause(expectedException);
+
+        assertThat(
+                        client.checkExists()
+                                .forPath(
+                                        
ZooKeeperStateHandleStore.getRootLockPath(pathInZooKeeper)))
+                .as("StateHandle should be marked for deletion.")
+                .isNull();
+        assertThat(client.checkExists().forPath(pathInZooKeeper))
+                .as("StateHandle should not be deleted, yet.")
+                .isNotNull();
+        assertThat(stateHandle.isDiscarded())
+                .as("The StateHandle should not be discarded, yet.")
+                .isFalse();
+
+        
assertThat(storeForDeletion.releaseAndTryRemove(pathInZooKeeper)).isTrue();
+        assertThat(client.checkExists().forPath(pathInZooKeeper))
+                .as("The StateHandle node should have been removed")
+                .isNull();
+        assertThat(stateHandle.isDiscarded())
+                .as("The StateHandle should have been discarded.")
+                .isTrue();
     }
 
     /**
@@ -310,10 +306,10 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * generic exception.
      */
     @Test
-    public void testFailingAddWithPossiblyInconsistentState() throws Exception 
{
+    void testFailingAddWithPossiblyInconsistentState() {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
-        CuratorFramework client = spy(ZOOKEEPER.getClient());
+        CuratorFramework client = spy(getZooKeeperClient());
         when(client.inTransaction()).thenThrow(new RuntimeException("Expected 
test Exception."));
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
@@ -323,44 +319,47 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         final String pathInZooKeeper = 
"/testAddDiscardStateHandleAfterFailure";
         final long state = 81282227L;
 
-        try {
-            // Test
-            store.addAndLock(
-                    pathInZooKeeper, new 
TestingLongStateHandleHelper.LongStateHandle(state));
-            fail("PossibleInconsistentStateException should have been 
thrown.");
-        } catch (PossibleInconsistentStateException ignored) {
-            // PossibleInconsistentStateException expected
-        }
+        assertThatExceptionOfType(PossibleInconsistentStateException.class)
+                .as("PossibleInconsistentStateException should have been 
thrown.")
+                .isThrownBy(
+                        () ->
+                                store.addAndLock(
+                                        pathInZooKeeper,
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(state)));
 
         // State handle created and not discarded
-        assertEquals(1, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(state, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
-        assertEquals(0, 
TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isOne();
+        
assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0)).isEqualTo(state);
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isZero();
     }
 
     @Test
-    public void testAddAndLockExistingNode() throws Exception {
+    void testAddAndLockExistingNode() throws Exception {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
-        final CuratorFramework client = ZOOKEEPER.getClient();
+        final CuratorFramework client = getZooKeeperClient();
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
                 new ZooKeeperStateHandleStore<>(client, stateHandleProvider);
         final String path = "/test";
         final long firstState = 1337L;
         final long secondState = 7331L;
         store.addAndLock(path, new 
TestingLongStateHandleHelper.LongStateHandle(firstState));
-        assertThrows(
-                StateHandleStore.AlreadyExistException.class,
-                () ->
-                        store.addAndLock(
-                                path,
-                                new 
TestingLongStateHandleHelper.LongStateHandle(secondState)));
+        assertThatExceptionOfType(StateHandleStore.AlreadyExistException.class)
+                .isThrownBy(
+                        () ->
+                                store.addAndLock(
+                                        path,
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(
+                                                secondState)));
         // There should be only single state handle from the first successful 
attempt.
-        assertEquals(1, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(firstState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isOne();
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0))
+                .isEqualTo(firstState);
         // No state should have been discarded.
-        assertEquals(0, 
TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isZero();
         // Get state handle from zookeeper.
-        assertEquals(firstState, 
store.getAndLock(path).retrieveState().getValue());
+        
assertThat(store.getAndLock(path).retrieveState().getValue()).isEqualTo(firstState);
     }
 
     /**
@@ -372,9 +371,9 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * @see <a 
href="https://issues.apache.org/jira/browse/CURATOR-584";>CURATOR-584</a>
      */
     @Test
-    public void testAddAndLockRetrySuccessfulTransaction() throws Exception {
+    void testAddAndLockRetrySuccessfulTransaction() throws Exception {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
-        final CuratorFramework client = ZOOKEEPER.getClient();
+        final CuratorFramework client = getZooKeeperClient();
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
                 new 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle>(
                         client, stateHandleProvider) {
@@ -391,65 +390,67 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         final long firstState = 1337L;
         store.addAndLock(path, new 
TestingLongStateHandleHelper.LongStateHandle(firstState));
         // There should be only single state handle from the first successful 
attempt.
-        assertEquals(1, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(firstState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isOne();
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0))
+                .isEqualTo(firstState);
         // No state should have been discarded.
-        assertEquals(0, 
TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isZero();
         // Get state handle from zookeeper.
-        assertEquals(firstState, 
store.getAndLock(path).retrieveState().getValue());
+        
assertThat(store.getAndLock(path).retrieveState().getValue()).isEqualTo(firstState);
     }
 
     @Test
-    public void testAddFailureHandlingForBadArgumentsException() {
+    void testAddFailureHandlingForBadArgumentsException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.BadArgumentsException());
     }
 
     @Test
-    public void testAddFailureHandlingForNoNodeException() {
+    void testAddFailureHandlingForNoNodeException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.NoNodeException());
     }
 
     @Test
-    public void testAddFailureHandlingForNoAuthException() {
+    void testAddFailureHandlingForNoAuthException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.NoAuthException());
     }
 
     @Test
-    public void testAddFailureHandlingForBadVersionException() {
+    void testAddFailureHandlingForBadVersionException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.BadVersionException());
     }
 
     @Test
-    public void testAddFailureHandlingForAuthFailedException() {
+    void testAddFailureHandlingForAuthFailedException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.AuthFailedException());
     }
 
     @Test
-    public void testAddFailureHandlingForInvalidACLException() {
+    void testAddFailureHandlingForInvalidACLException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.InvalidACLException());
     }
 
     @Test
-    public void testAddFailureHandlingForSessionMovedException() {
+    void testAddFailureHandlingForSessionMovedException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.SessionMovedException());
     }
 
     @Test
-    public void testAddFailureHandlingForNotReadOnlyException() {
+    void testAddFailureHandlingForNotReadOnlyException() {
         testFailingAddWithStateDiscardTriggeredFor(new 
KeeperException.NotReadOnlyException());
     }
 
-    private static void testFailingAddWithStateDiscardTriggeredFor(Exception 
actualException) {
+    private void testFailingAddWithStateDiscardTriggeredFor(Exception 
actualException) {
         testFailingAddWithStateDiscardTriggeredFor(actualException, 
actualException.getClass());
     }
 
-    private static void testFailingAddWithStateDiscardTriggeredFor(
+    private void testFailingAddWithStateDiscardTriggeredFor(
             Exception actualException, Class<? extends Throwable> 
expectedException) {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
                 new 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle>(
-                        ZOOKEEPER.getClient(), stateHandleProvider) {
+                        getZooKeeperClient(), stateHandleProvider) {
                     @Override
                     protected void writeStoreHandleTransactionally(
                             String path, byte[] serializedStoreHandle) throws 
Exception {
@@ -462,29 +463,29 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
                 "/testAddDiscardStateHandleAfterFailure-" + 
expectedException.getSimpleName();
         final long state = 81282227L;
 
-        try {
-            // Test
-            store.addAndLock(
-                    pathInZooKeeper, new 
TestingLongStateHandleHelper.LongStateHandle(state));
-            fail(expectedException.getSimpleName() + " should have been 
thrown.");
-        } catch (Exception ex) {
-            assertThat(ex, IsInstanceOf.instanceOf(expectedException));
-        }
+        assertThatExceptionOfType(expectedException)
+                .as(expectedException.getSimpleName() + " should have been 
thrown.")
+                .isThrownBy(
+                        () ->
+                                store.addAndLock(
+                                        pathInZooKeeper,
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(state)));
 
         // State handle created and discarded
-        assertEquals(1, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(state, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
-        assertEquals(1, 
TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isOne();
+        
assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0)).isEqualTo(state);
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isOne();
     }
 
     /** Tests that a state handle is replaced. */
     @Test
-    public void testReplace() throws Exception {
+    void testReplace() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
         // Config
         final String pathInZooKeeper = "/testReplace";
@@ -501,33 +502,35 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         // Verify
         // State handles created
-        assertEquals(2, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(initialState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
-        assertEquals(replaceState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(1));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isEqualTo(2);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0))
+                .isEqualTo(initialState);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(1))
+                .isEqualTo(replaceState);
 
         // Path created and is persistent
-        Stat stat = 
ZOOKEEPER.getClient().checkExists().forPath(pathInZooKeeper);
-        assertNotNull(stat);
-        assertEquals(0, stat.getEphemeralOwner());
+        Stat stat = 
getZooKeeperClient().checkExists().forPath(pathInZooKeeper);
+        assertThat(stat).isNotNull();
+        assertThat(stat.getEphemeralOwner()).isZero();
 
         // Data is equal
         @SuppressWarnings("unchecked")
         final long actual =
                 
((RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>)
                                 InstantiationUtil.deserializeObject(
-                                        
ZOOKEEPER.getClient().getData().forPath(pathInZooKeeper),
+                                        
getZooKeeperClient().getData().forPath(pathInZooKeeper),
                                         ClassLoader.getSystemClassLoader()))
                         .retrieveState()
                         .getValue();
 
-        assertEquals(replaceState, actual);
+        assertThat(actual).isEqualTo(replaceState);
     }
 
     @Test
-    public void testReplaceRequiringALock() throws Exception {
+    void testReplaceRequiringALock() throws Exception {
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testReplaceOnMarkedForDeletionNode");
+                        getZooKeeperClient(), 
"/testReplaceOnMarkedForDeletionNode");
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
                 new ZooKeeperStateHandleStore<>(client, new 
TestingLongStateHandleHelper());
 
@@ -545,33 +548,35 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         final TestingLongStateHandleHelper.LongStateHandle updatedStateHandle =
                 new TestingLongStateHandleHelper.LongStateHandle(updatedState);
 
-        assertThrows(
-                IllegalStateException.class,
-                () -> zkStore.replace(path, versionBeforeDeletion, 
updatedStateHandle));
+        assertThatThrownBy(() -> zkStore.replace(path, versionBeforeDeletion, 
updatedStateHandle))
+                .isInstanceOf(IllegalStateException.class);
     }
 
     /** Tests that a non existing path throws an Exception. */
-    @Test(expected = Exception.class)
-    public void testReplaceNonExistingPath() throws Exception {
+    @Test
+    void testReplaceNonExistingPath() {
         final 
RetrievableStateStorageHelper<TestingLongStateHandleHelper.LongStateHandle>
                 stateStorage = new TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateStorage);
-
-        store.replace(
-                "/testReplaceNonExistingPath",
-                IntegerResourceVersion.valueOf(0),
-                new TestingLongStateHandleHelper.LongStateHandle(1L));
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateStorage);
+
+        assertThatThrownBy(
+                        () ->
+                                store.replace(
+                                        "/testReplaceNonExistingPath",
+                                        IntegerResourceVersion.valueOf(0),
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(1L)))
+                .isInstanceOf(Exception.class);
     }
 
     /** Tests that the replace state handle is discarded if ZooKeeper setData 
fails. */
     @Test
-    public void testReplaceDiscardStateHandleAfterFailure() throws Exception {
+    void testReplaceDiscardStateHandleAfterFailure() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
-        CuratorFramework client = spy(ZOOKEEPER.getClient());
+        CuratorFramework client = spy(getZooKeeperClient());
         when(client.setData()).thenThrow(new RuntimeException("Expected test 
Exception."));
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
@@ -586,95 +591,99 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         store.addAndLock(
                 pathInZooKeeper, new 
TestingLongStateHandleHelper.LongStateHandle(initialState));
 
-        try {
-            store.replace(
-                    pathInZooKeeper,
-                    IntegerResourceVersion.valueOf(0),
-                    new 
TestingLongStateHandleHelper.LongStateHandle(replaceState));
-            fail("Did not throw expected exception");
-        } catch (Exception ignored) {
-        }
+        assertThatExceptionOfType(Exception.class)
+                .as("Did not throw expected exception")
+                .isThrownBy(
+                        () ->
+                                store.replace(
+                                        pathInZooKeeper,
+                                        IntegerResourceVersion.valueOf(0),
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(
+                                                replaceState)));
 
         // Verify
         // State handle created and discarded
-        assertEquals(2, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(initialState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
-        assertEquals(replaceState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(1));
-        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0),
 is(0));
-        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(1),
 is(0));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isEqualTo(2);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0))
+                .isEqualTo(initialState);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(1))
+                .isEqualTo(replaceState);
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isZero();
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(1))
+                .isZero();
 
         // Initial value
         @SuppressWarnings("unchecked")
         final long actual =
                 
((RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>)
                                 InstantiationUtil.deserializeObject(
-                                        
ZOOKEEPER.getClient().getData().forPath(pathInZooKeeper),
+                                        
getZooKeeperClient().getData().forPath(pathInZooKeeper),
                                         ClassLoader.getSystemClassLoader()))
                         .retrieveState()
                         .getValue();
 
-        assertEquals(initialState, actual);
+        assertThat(actual).isEqualTo(initialState);
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithNoNodeException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithNoNodeException() throws Exception {
         testDiscardAfterReplaceFailureWith(
                 new KeeperException.NoNodeException(), 
StateHandleStore.NotExistException.class);
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithNodeExistsException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithNodeExistsException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.NodeExistsException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithBadArgumentsException() 
throws Exception {
+    void testDiscardAfterReplaceFailureWithBadArgumentsException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.BadArgumentsException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithNoAuthException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithNoAuthException() throws Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.NoAuthException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithBadVersionException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithBadVersionException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.BadVersionException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithAuthFailedException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithAuthFailedException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.AuthFailedException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithInvalidACLException() throws 
Exception {
+    void testDiscardAfterReplaceFailureWithInvalidACLException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.InvalidACLException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithSessionMovedException() 
throws Exception {
+    void testDiscardAfterReplaceFailureWithSessionMovedException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.SessionMovedException());
     }
 
     @Test
-    public void testDiscardAfterReplaceFailureWithNotReadOnlyException() 
throws Exception {
+    void testDiscardAfterReplaceFailureWithNotReadOnlyException() throws 
Exception {
         testDiscardAfterReplaceFailureWith(new 
KeeperException.NotReadOnlyException());
     }
 
-    private static void testDiscardAfterReplaceFailureWith(Exception 
actualException)
-            throws Exception {
+    private void testDiscardAfterReplaceFailureWith(Exception actualException) 
throws Exception {
         testDiscardAfterReplaceFailureWith(actualException, 
actualException.getClass());
     }
 
-    private static void testDiscardAfterReplaceFailureWith(
-            Exception actualException, Class<? extends Throwable> 
expectedException)
+    private void testDiscardAfterReplaceFailureWith(
+            Exception actualException, Class<? extends Throwable> 
expectedExceptionClass)
             throws Exception {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
                 new 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle>(
-                        ZOOKEEPER.getClient(), stateHandleProvider) {
+                        getZooKeeperClient(), stateHandleProvider) {
                     @Override
                     protected void setStateHandle(
                             String path, byte[] serializedStateHandle, int 
expectedVersion)
@@ -685,7 +694,8 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         // Config
         final String pathInZooKeeper =
-                "/testReplaceDiscardStateHandleAfterFailure-" + 
expectedException.getSimpleName();
+                "/testReplaceDiscardStateHandleAfterFailure-"
+                        + expectedExceptionClass.getSimpleName();
         final long initialState = 30968470898L;
         final long replaceState = 88383776661L;
 
@@ -693,66 +703,70 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         store.addAndLock(
                 pathInZooKeeper, new 
TestingLongStateHandleHelper.LongStateHandle(initialState));
 
-        try {
-            store.replace(
-                    pathInZooKeeper,
-                    IntegerResourceVersion.valueOf(0),
-                    new 
TestingLongStateHandleHelper.LongStateHandle(replaceState));
-            fail("Did not throw expected exception");
-        } catch (Throwable t) {
-            assertThat(t, IsInstanceOf.instanceOf(expectedException));
-        }
+        assertThatThrownBy(
+                        () ->
+                                store.replace(
+                                        pathInZooKeeper,
+                                        IntegerResourceVersion.valueOf(0),
+                                        new 
TestingLongStateHandleHelper.LongStateHandle(
+                                                replaceState)))
+                .as("Did not throw expected exception")
+                .isInstanceOf(expectedExceptionClass);
 
         // State handle created and discarded
-        assertEquals(2, TestingLongStateHandleHelper.getGlobalStorageSize());
-        assertEquals(initialState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(0));
-        assertEquals(replaceState, 
TestingLongStateHandleHelper.getStateHandleValueByIndex(1));
-        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0),
 is(0));
-        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(1),
 is(1));
+        
assertThat(TestingLongStateHandleHelper.getGlobalStorageSize()).isEqualTo(2);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(0))
+                .isEqualTo(initialState);
+        assertThat(TestingLongStateHandleHelper.getStateHandleValueByIndex(1))
+                .isEqualTo(replaceState);
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(0))
+                .isZero();
+        
assertThat(TestingLongStateHandleHelper.getDiscardCallCountForStateHandleByIndex(1))
+                .isOne();
 
         // Initial value
         @SuppressWarnings("unchecked")
         final long actual =
                 
((RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>)
                                 InstantiationUtil.deserializeObject(
-                                        
ZOOKEEPER.getClient().getData().forPath(pathInZooKeeper),
+                                        
getZooKeeperClient().getData().forPath(pathInZooKeeper),
                                         ClassLoader.getSystemClassLoader()))
                         .retrieveState()
                         .getValue();
 
-        assertEquals(initialState, actual);
+        assertThat(actual).isEqualTo(initialState);
     }
 
     /** Tests get operation. */
     @Test
-    public void testGetAndExists() throws Exception {
+    void testGetAndExists() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
         // Config
         final String pathInZooKeeper = "/testGetAndExists";
         final long state = 311222268470898L;
 
         // Test
-        assertThat(store.exists(pathInZooKeeper).isExisting(), is(false));
+        assertThat(store.exists(pathInZooKeeper).isExisting()).isFalse();
 
         store.addAndLock(pathInZooKeeper, new 
TestingLongStateHandleHelper.LongStateHandle(state));
         RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle> 
actual =
                 store.getAndLock(pathInZooKeeper);
 
         // Verify
-        assertEquals(state, actual.retrieveState().getValue());
-        assertTrue(store.exists(pathInZooKeeper).getValue() >= 0);
+        assertThat(actual.retrieveState().getValue()).isEqualTo(state);
+        
assertThat(store.exists(pathInZooKeeper).getValue()).isGreaterThanOrEqualTo(0);
     }
 
     @Test
-    public void testExistsOnMarkedForDeletionNode() throws Exception {
+    void testExistsOnMarkedForDeletionNode() throws Exception {
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testExistsOnMarkedForDeletionEntry");
+                        getZooKeeperClient(), 
"/testExistsOnMarkedForDeletionEntry");
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
                 new ZooKeeperStateHandleStore<>(client, new 
TestingLongStateHandleHelper());
 
@@ -763,28 +777,30 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         markNodeForDeletion(client, markedForDeletionPath);
 
-        
assertFalse(zkStore.exists(generateZookeeperPath(markedForDeletionPath)).isExisting());
+        
assertThat(zkStore.exists(generateZookeeperPath(markedForDeletionPath)).isExisting())
+                .isFalse();
     }
 
     /** Tests that a non existing path throws an Exception. */
-    @Test(expected = Exception.class)
-    public void testGetNonExistingPath() throws Exception {
+    @Test
+    void testGetNonExistingPath() {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
-        store.getAndLock("/testGetNonExistingPath");
+        assertThatThrownBy(() -> store.getAndLock("/testGetNonExistingPath"))
+                .isInstanceOf(Exception.class);
     }
 
     /** Tests that all added state is returned. */
     @Test
-    public void testGetAll() throws Exception {
+    void testGetAll() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
         // Config
         final String pathInZooKeeper = "/testGetAll";
@@ -803,17 +819,17 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         for 
(Tuple2<RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>, 
String>
                 val : store.getAllAndLock()) {
-            assertTrue(expected.remove(val.f0.retrieveState().getValue()));
+            
assertThat(expected.remove(val.f0.retrieveState().getValue())).isTrue();
         }
-        assertEquals(0, expected.size());
+        assertThat(expected).isEmpty();
     }
 
     @Test
-    public void testGetAllAndLockOnConcurrentDelete() throws Exception {
+    void testGetAllAndLockOnConcurrentDelete() throws Exception {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testGetAllAndLockOnConcurrentDelete");
+                        getZooKeeperClient(), 
"/testGetAllAndLockOnConcurrentDelete");
 
         // this store simulates the ZooKeeper connection for maintaining the 
lifecycle (i.e.
         // creating and deleting the nodes) of the StateHandles
@@ -861,18 +877,17 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
                                     return childNodes;
                                 });
 
-        assertEquals(
-                "Only the StateHandle that was expected to be kept should be 
returned.",
-                stateToKeep,
-                
Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue());
+        
assertThat(Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue())
+                .as("Only the StateHandle that was expected to be kept should 
be returned.")
+                .isEqualTo(stateToKeep);
     }
 
     @Test
-    public void testGetAllAndLockWhileEntryIsMarkedForDeletion() throws 
Exception {
+    void testGetAllAndLockWhileEntryIsMarkedForDeletion() throws Exception {
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testGetAllAndLockWhileEntryIsMarkedForDeletion");
+                        getZooKeeperClient(), 
"/testGetAllAndLockWhileEntryIsMarkedForDeletion");
 
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle>
                 stateHandleStore = new ZooKeeperStateHandleStore<>(client, 
stateHandleProvider);
@@ -901,20 +916,19 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
                                 String>>
                 actuallyLockedHandles = stateHandleStore.getAllAndLock();
 
-        assertEquals(
-                "Only the StateHandle that was expected to be kept should be 
returned.",
-                stateToKeep,
-                
Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue());
+        
assertThat(Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue())
+                .as("Only the StateHandle that was expected to be kept should 
be returned.")
+                .isEqualTo(stateToKeep);
     }
 
     /** Tests that the state is returned sorted. */
     @Test
-    public void testGetAllSortedByName() throws Exception {
+    void testGetAllSortedByName() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
         // Config
         final String basePath = "/testGetAllSortedByName";
@@ -930,25 +944,25 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         
List<Tuple2<RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>,
 String>>
                 actual = store.getAllAndLock();
-        assertEquals(expected.length, actual.size());
+        assertThat(actual).hasSize(expected.length);
 
         // bring the elements in sort order
         Arrays.sort(expected);
-        Collections.sort(actual, Comparator.comparing(o -> o.f1));
+        actual.sort(Comparator.comparing(o -> o.f1));
 
         for (int i = 0; i < expected.length; i++) {
-            assertEquals(expected[i], (Long) 
actual.get(i).f0.retrieveState().getValue());
+            assertThat((Long) 
actual.get(i).f0.retrieveState().getValue()).isEqualTo(expected[i]);
         }
     }
 
     /** Tests that state handles are correctly removed. */
     @Test
-    public void testRemove() throws Exception {
+    void testRemove() throws Exception {
         // Setup
         final TestingLongStateHandleHelper stateHandleProvider = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateHandleProvider);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateHandleProvider);
 
         // Config
         final String pathInZooKeeper = "/testRemove";
@@ -962,10 +976,9 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         store.releaseAndTryRemove(pathInZooKeeper);
 
         // Verify discarded
-        assertEquals(0, 
ZOOKEEPER.getClient().getChildren().forPath("/").size());
-        assertEquals(
-                numberOfGlobalDiscardCalls + 1,
-                TestingLongStateHandleHelper.getGlobalDiscardCount());
+        assertThat(getZooKeeperClient().getChildren().forPath("/")).isEmpty();
+        assertThat(TestingLongStateHandleHelper.getGlobalDiscardCount())
+                .isEqualTo(numberOfGlobalDiscardCalls + 1);
     }
 
     /**
@@ -973,11 +986,11 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * remove the respective ZooKeeper ZNodes.
      */
     @Test
-    public void testCorruptedData() throws Exception {
+    void testCorruptedData() throws Exception {
         final TestingLongStateHandleHelper stateStorage = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> store =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
stateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
stateStorage);
 
         final Collection<Long> input = new HashSet<>();
         input.add(1L);
@@ -989,7 +1002,7 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         }
 
         // corrupt one of the entries
-        ZOOKEEPER.getClient().setData().forPath("/" + 2, new byte[2]);
+        getZooKeeperClient().setData().forPath("/" + 2, new byte[2]);
 
         
List<Tuple2<RetrievableStateHandle<TestingLongStateHandleHelper.LongStateHandle>,
 String>>
                 allEntries = store.getAllAndLock();
@@ -1004,7 +1017,7 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
             actual.add(entry.f0.retrieveState().getValue());
         }
 
-        assertEquals(expected, actual);
+        assertThat(actual).isEqualTo(expected);
     }
 
     /**
@@ -1014,14 +1027,14 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * on the specified node.
      */
     @Test
-    public void testConcurrentDeleteOperation() throws Exception {
+    void testConcurrentDeleteOperation() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
zkStore1 =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
zkStore2 =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         final String statePath = "/state";
 
@@ -1034,21 +1047,21 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         zkStore1.releaseAndTryRemove(statePath);
 
         // sanity check
-        assertEquals(42L, stateHandle.retrieveState().getValue());
+        assertThat(stateHandle.retrieveState().getValue()).isEqualTo(42L);
 
-        Stat nodeStat = ZOOKEEPER.getClient().checkExists().forPath(statePath);
+        Stat nodeStat = getZooKeeperClient().checkExists().forPath(statePath);
 
-        assertNotNull(
-                "NodeStat should not be null, otherwise the referenced node 
does not exist.",
-                nodeStat);
+        assertThat(nodeStat)
+                .as("NodeStat should not be null, otherwise the referenced 
node does not exist.")
+                .isNotNull();
 
         zkStore2.releaseAndTryRemove(statePath);
 
-        nodeStat = ZOOKEEPER.getClient().checkExists().forPath(statePath);
+        nodeStat = getZooKeeperClient().checkExists().forPath(statePath);
 
-        assertNull(
-                "NodeState should be null, because the referenced node should 
no longer exist.",
-                nodeStat);
+        assertThat(nodeStat)
+                .as("NodeState should be null, because the referenced node 
should no longer exist.")
+                .isNull();
     }
 
     /**
@@ -1058,14 +1071,14 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * retrieved (e.g. deserialization problem).
      */
     @Test
-    public void testLockCleanupWhenGetAndLockFails() throws Exception {
+    void testLockCleanupWhenGetAndLockFails() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
zkStore1 =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> 
zkStore2 =
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         final String path = "/state";
 
@@ -1074,33 +1087,30 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
         final byte[] corruptedData = {1, 2};
 
         // corrupt the data
-        ZOOKEEPER.getClient().setData().forPath(path, corruptedData);
+        getZooKeeperClient().setData().forPath(path, corruptedData);
 
-        try {
-            zkStore2.getAndLock(path);
-            fail("Should fail because we cannot deserialize the node's data");
-        } catch (IOException ignored) {
-            // expected to fail
-        }
+        assertThatExceptionOfType(IOException.class)
+                .as("Should fail because we cannot deserialize the node's 
data")
+                .isThrownBy(() -> zkStore2.getAndLock(path));
 
         // check that there is no lock node left
         String lockNodePath = zkStore2.getInstanceLockPath(path);
 
-        Stat stat = ZOOKEEPER.getClient().checkExists().forPath(lockNodePath);
+        Stat stat = getZooKeeperClient().checkExists().forPath(lockNodePath);
 
         // zkStore2 should not have created a lock node
-        assertNull("zkStore2 should not have created a lock node.", stat);
+        assertThat(stat).as("zkStore2 should not have created a lock 
node.").isNull();
 
-        Collection<String> children = 
ZOOKEEPER.getClient().getChildren().forPath(path);
+        Collection<String> children = 
getZooKeeperClient().getChildren().forPath(path);
 
         // there should be exactly one lock node from zkStore1
-        assertEquals(1, children.size());
+        assertThat(children).hasSize(1);
 
         zkStore1.releaseAndTryRemove(path);
 
-        stat = ZOOKEEPER.getClient().checkExists().forPath(path);
+        stat = getZooKeeperClient().checkExists().forPath(path);
 
-        assertNull("The state node should have been removed.", stat);
+        assertThat(stat).as("The state node should have been 
removed.").isNull();
     }
 
     /**
@@ -1109,12 +1119,12 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * <p>Tests that lock nodes will be released if the client dies.
      */
     @Test
-    public void testLockCleanupWhenClientTimesOut() throws Exception {
+    void testLockCleanupWhenClientTimesOut() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
 
         Configuration configuration = new Configuration();
         configuration.setString(
-                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
ZOOKEEPER.getConnectString());
+                HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, 
zooKeeperExtension.getConnectString());
         
configuration.setInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT, 
100);
         configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT, 
"timeout");
 
@@ -1140,13 +1150,13 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
             Stat stat = client2.checkExists().forPath(path);
 
             // check that our state node still exists
-            assertNotNull(stat);
+            assertThat(stat).isNotNull();
 
             Collection<String> children =
-                    
client2.getChildren().forPath(zkStore.getRootLockPath(path));
+                    
client2.getChildren().forPath(ZooKeeperStateHandleStore.getRootLockPath(path));
 
             // check that the lock node has been released
-            assertEquals(0, children.size());
+            assertThat(children).isEmpty();
         }
     }
 
@@ -1156,11 +1166,11 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * <p>Tests that we can release a locked state handles in the 
ZooKeeperStateHandleStore.
      */
     @Test
-    public void testRelease() throws Exception {
+    void testRelease() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         final String path = "/state";
 
@@ -1168,26 +1178,25 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         final String lockPath = zkStore.getInstanceLockPath(path);
 
-        Stat stat = ZOOKEEPER.getClient().checkExists().forPath(lockPath);
+        Stat stat = getZooKeeperClient().checkExists().forPath(lockPath);
 
-        assertNotNull("Expected an existing lock", stat);
+        assertThat(stat).as("Expected an existing lock").isNotNull();
 
         zkStore.release(path);
 
         stat =
-                ZOOKEEPER
-                        .getClient()
+                getZooKeeperClient()
                         .checkExists()
                         
.forPath(ZooKeeperStateHandleStore.getRootLockPath(path));
 
         // release should have removed the lock child
-        assertEquals("Expected no lock nodes as children", 0, 
stat.getNumChildren());
+        assertThat(stat.getNumChildren()).as("Expected no lock nodes as 
children").isZero();
 
         zkStore.releaseAndTryRemove(path);
 
-        stat = ZOOKEEPER.getClient().checkExists().forPath(path);
+        stat = getZooKeeperClient().checkExists().forPath(path);
 
-        assertNull("State node should have been removed.", stat);
+        assertThat(stat).as("State node should have been removed.").isNull();
     }
 
     /**
@@ -1196,11 +1205,11 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
      * <p>Tests that we can release all locked state handles in the 
ZooKeeperStateHandleStore.
      */
     @Test
-    public void testReleaseAll() throws Exception {
+    void testReleaseAll() throws Exception {
         final TestingLongStateHandleHelper longStateStorage = new 
TestingLongStateHandleHelper();
 
         
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
-                new ZooKeeperStateHandleStore<>(ZOOKEEPER.getClient(), 
longStateStorage);
+                new ZooKeeperStateHandleStore<>(getZooKeeperClient(), 
longStateStorage);
 
         final Collection<String> paths = Arrays.asList("/state1", "/state2", 
"/state3");
 
@@ -1210,42 +1219,41 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         for (String path : paths) {
             Stat stat =
-                    
ZOOKEEPER.getClient().checkExists().forPath(zkStore.getInstanceLockPath(path));
+                    
getZooKeeperClient().checkExists().forPath(zkStore.getInstanceLockPath(path));
 
-            assertNotNull("Expecte and existing lock.", stat);
+            assertThat(stat).as("Expecte and existing lock.").isNotNull();
         }
 
         zkStore.releaseAll();
 
         for (String path : paths) {
             Stat stat =
-                    ZOOKEEPER
-                            .getClient()
+                    getZooKeeperClient()
                             .checkExists()
                             
.forPath(ZooKeeperStateHandleStore.getRootLockPath(path));
 
-            assertEquals(0, stat.getNumChildren());
+            assertThat(stat.getNumChildren()).isZero();
         }
     }
 
     @Test
-    public void testRemoveAllHandlesShouldRemoveAllPaths() throws Exception {
+    void testRemoveAllHandlesShouldRemoveAllPaths() throws Exception {
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
                 new ZooKeeperStateHandleStore<>(
-                        
ZooKeeperUtils.useNamespaceAndEnsurePath(ZOOKEEPER.getClient(), "/path"),
+                        
ZooKeeperUtils.useNamespaceAndEnsurePath(getZooKeeperClient(), "/path"),
                         new TestingLongStateHandleHelper());
 
         zkStore.addAndLock("/state", new 
TestingLongStateHandleHelper.LongStateHandle(1L));
         zkStore.clearEntries();
 
-        assertThat(zkStore.getAllHandles(), is(empty()));
+        assertThat(zkStore.getAllHandles()).isEmpty();
     }
 
     @Test
-    public void testGetAllHandlesWithMarkedForDeletionEntries() throws 
Exception {
+    void testGetAllHandlesWithMarkedForDeletionEntries() throws Exception {
         final CuratorFramework client =
                 ZooKeeperUtils.useNamespaceAndEnsurePath(
-                        ZOOKEEPER.getClient(), 
"/testGetAllHandlesWithMarkedForDeletionEntries");
+                        getZooKeeperClient(), 
"/testGetAllHandlesWithMarkedForDeletionEntries");
         final 
ZooKeeperStateHandleStore<TestingLongStateHandleHelper.LongStateHandle> zkStore 
=
                 new ZooKeeperStateHandleStore<>(client, new 
TestingLongStateHandleHelper());
 
@@ -1260,10 +1268,8 @@ public class ZooKeeperStateHandleStoreTest extends 
TestLogger {
 
         markNodeForDeletion(client, markedForDeletionNodeName);
 
-        assertThat(
-                zkStore.getAllHandles(),
-                IsIterableContainingInAnyOrder.containsInAnyOrder(
-                        notMarkedForDeletionNodeName, 
markedForDeletionNodeName));
+        assertThat(zkStore.getAllHandles())
+                .containsExactlyInAnyOrder(notMarkedForDeletionNodeName, 
markedForDeletionNodeName);
     }
 
     private static void markNodeForDeletion(CuratorFramework client, String 
childNodeName)


Reply via email to