From cea4347c3ea17260173cb18c06d0807af08e5923 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 22 Mar 2022 12:58:52 +0100 Subject: [PATCH 1/2] [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 5f303e1bcc897..c18408f08d575 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.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 void testRecoveredAfterConnectionLoss() throws Exception { 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(); } From 8f24707016208c4695c8e35ffa9150111464296a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 22 Mar 2022 13:03:42 +0100 Subject: [PATCH 2/2] [hotfix][runtime] Adds debug logs for FinalRequestProcessor to CI run org.apache.zookeeper.server.FinalRequestProcessor provides debug logs for actual operations being performed on the server, which might be interesting to see when investigating test instabilities. The following logs are part of the debug logs: Processing request:: sessionid:0x1003757ccd40000 type:getData cxid:0x11 zxid:0xfffffffffffffffe txntype:unknown reqpath:/flink/default/checkpoint_id_counter ...contains the type of request (e.g. 'getData') and path (e.g. '/flink/default/checkpoint_id_counter'). --- tools/ci/log4j.properties | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index bf831743e6f26..80a611405d7e4 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -61,6 +61,11 @@ logger.zookeeper.name = org.apache.zookeeper logger.zookeeper.level = INFO logger.zookeeper.additivity = false logger.zookeeper.appenderRef.zk.ref = ZooKeeperServerAppender +# FinalRequestProcessor in DEBUG mode enables us to see requests being processed by the ZK server +logger.zookeeper_FinalRequestProcessor.name = org.apache.zookeeper.server.FinalRequestProcessor +logger.zookeeper_FinalRequestProcessor.level = DEBUG +logger.zookeeper_FinalRequestProcessor.additivity = false +logger.zookeeper_FinalRequestProcessor.appenderRef.zk.ref = ZooKeeperServerAppender logger.shaded_zookeeper.name = org.apache.flink.shaded.zookeeper3 logger.shaded_zookeeper.level = INFO logger.shaded_zookeeper.additivity = false