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

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

commit 7adbdddff5d3eee00a150d672fe4c6f36e4ad559
Author: Matthias Pohl <[email protected]>
AuthorDate: Tue Mar 22 12:58:52 2022 +0100

    [FLINK-26797][runtime] Makes the test less strict
    
    We only care about monotonically increasing numbers.
    Unstable connections could cause a retry by the curator
    client. See FLINK-26120 where a the same error cause was
    fixed in the same way.
---
 .../ZKCheckpointIDCounterMultiServersTest.java           | 16 +++-------------
 1 file changed, 3 insertions(+), 13 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java
index e474de3..45f2c95 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java
@@ -33,9 +33,7 @@ import 
org.apache.flink.shaded.curator4.org.apache.curator.framework.state.Conne
 import org.junit.Rule;
 import org.junit.Test;
 
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.number.OrderingComparison.greaterThan;
 import static org.junit.Assert.assertThat;
 
 /** Tests for {@link ZooKeeperCheckpointIDCounter} in a ZooKeeper ensemble. */
@@ -71,22 +69,14 @@ public final class ZKCheckpointIDCounterMultiServersTest 
extends TestLogger {
                             curatorFrameworkWrapper.asCuratorFramework(), 
listener);
             idCounter.start();
 
-            AtomicLong localCounter = new AtomicLong(1L);
-
-            assertThat(
-                    "ZooKeeperCheckpointIDCounter doesn't properly work.",
-                    idCounter.getAndIncrement(),
-                    is(localCounter.getAndIncrement()));
+            final long initialID = idCounter.getAndIncrement();
 
             zooKeeperResource.restart();
 
             connectionLossLatch.await();
             reconnectedLatch.await();
 
-            assertThat(
-                    "ZooKeeperCheckpointIDCounter doesn't properly work after 
reconnected.",
-                    idCounter.getAndIncrement(),
-                    is(localCounter.getAndIncrement()));
+            assertThat(idCounter.getAndIncrement(), greaterThan(initialID));
         } finally {
             curatorFrameworkWrapper.close();
         }

Reply via email to