This is an automated email from the ASF dual-hosted git repository. jqin pushed a commit to branch release-1.11 in repository https://gitbox.apache.org/repos/asf/flink.git
commit c3c2babd287fbf0c817a8b91c4c88d10d1b49aa3 Author: Stephan Ewen <[email protected]> AuthorDate: Fri Nov 27 14:53:28 2020 +0100 [hotfix][tests] Pull nested test suites out of CheckpointIDCounterTest and follow standard test-base pattern. That makes the tests consistent with the common practice in Flink and also ensures that the tests are picked up by the Maven/Surefire build. Previously, these tests were not executed during a Maven build. --- ...rTest.java => CheckpointIDCounterTestBase.java} | 88 +++------------------- .../StandaloneCheckpointIDCounterTest.java | 31 ++++++++ .../ZooKeeperCheckpointIDCounterITCase.java | 88 ++++++++++++++++++++++ 3 files changed, 129 insertions(+), 78 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java similarity index 64% rename from flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java index e8b4f22..3323e2c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java @@ -19,13 +19,8 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; - -import org.junit.AfterClass; -import org.junit.Before; import org.junit.Test; import java.util.ArrayList; @@ -39,73 +34,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -public abstract class CheckpointIDCounterTest extends TestLogger { - - protected abstract CheckpointIDCounter createCompletedCheckpoints() throws Exception; - - public static class StandaloneCheckpointIDCounterTest extends CheckpointIDCounterTest { - - @Override - protected CheckpointIDCounter createCompletedCheckpoints() throws Exception { - return new StandaloneCheckpointIDCounter(); - } - } - public static class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTest { - - private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); - - @AfterClass - public static void tearDown() throws Exception { - ZooKeeper.shutdown(); - } - - @Before - public void cleanUp() throws Exception { - ZooKeeper.deleteAll(); - } - - /** - * Tests that counter node is removed from ZooKeeper after shutdown. - */ - @Test - public void testShutdownRemovesState() throws Exception { - CheckpointIDCounter counter = createCompletedCheckpoints(); - counter.start(); - - CuratorFramework client = ZooKeeper.getClient(); - assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); - - counter.shutdown(JobStatus.FINISHED); - assertNull(client.checkExists().forPath("/checkpoint-id-counter")); - } - - /** - * Tests that counter node is NOT removed from ZooKeeper after suspend. - */ - @Test - public void testSuspendKeepsState() throws Exception { - CheckpointIDCounter counter = createCompletedCheckpoints(); - counter.start(); - - CuratorFramework client = ZooKeeper.getClient(); - assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); +/** + * Test base class with common tests for the {@link CheckpointIDCounter} implementations. + */ +public abstract class CheckpointIDCounterTestBase extends TestLogger { - counter.shutdown(JobStatus.SUSPENDED); - assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); - } - - @Override - protected CheckpointIDCounter createCompletedCheckpoints() throws Exception { - return new ZooKeeperCheckpointIDCounter( - ZooKeeper.getClient(), - "/checkpoint-id-counter", - new DefaultLastStateConnectionStateListener()); - } - } + protected abstract CheckpointIDCounter createCheckpointIdCounter() throws Exception; // --------------------------------------------------------------------------------------------- @@ -114,7 +49,7 @@ public abstract class CheckpointIDCounterTest extends TestLogger { */ @Test public void testSerialIncrementAndGet() throws Exception { - final CheckpointIDCounter counter = createCompletedCheckpoints(); + final CheckpointIDCounter counter = createCheckpointIdCounter(); try { counter.start(); @@ -143,7 +78,7 @@ public abstract class CheckpointIDCounterTest extends TestLogger { // Setup final CountDownLatch startLatch = new CountDownLatch(1); - final CheckpointIDCounter counter = createCompletedCheckpoints(); + final CheckpointIDCounter counter = createCheckpointIdCounter(); counter.start(); ExecutorService executor = null; @@ -166,10 +101,7 @@ public abstract class CheckpointIDCounterTest extends TestLogger { // Get the counts for (Future<List<Long>> result : resultFutures) { List<Long> counts = result.get(); - - for (long val : counts) { - all.add(val); - } + all.addAll(counts); } // Verify @@ -201,7 +133,7 @@ public abstract class CheckpointIDCounterTest extends TestLogger { */ @Test public void testSetCount() throws Exception { - final CheckpointIDCounter counter = createCompletedCheckpoints(); + final CheckpointIDCounter counter = createCheckpointIdCounter(); counter.start(); // Test setCount @@ -220,7 +152,7 @@ public abstract class CheckpointIDCounterTest extends TestLogger { private static class Incrementer implements Callable<List<Long>> { /** Total number of {@link CheckpointIDCounter#getAndIncrement()} calls. */ - private final static int NumIncrements = 128; + private static final int NumIncrements = 128; private final CountDownLatch startLatch; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounterTest.java new file mode 100644 index 0000000..5c296e2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounterTest.java @@ -0,0 +1,31 @@ +/* + * 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.checkpoint; + +/** + * Unit tests for the {@link StandaloneCheckpointIDCounter}. + * The tests are inherited from the test base class {@link CheckpointIDCounterTestBase}. + */ +public class StandaloneCheckpointIDCounterTest extends CheckpointIDCounterTestBase { + + @Override + protected CheckpointIDCounter createCheckpointIdCounter() throws Exception { + return new StandaloneCheckpointIDCounter(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java new file mode 100644 index 0000000..d86d55c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java @@ -0,0 +1,88 @@ +/* + * 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.checkpoint; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; + +import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; + +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * Unit tests for the {@link ZooKeeperCheckpointIDCounter}. + * The tests are inherited from the test base class {@link CheckpointIDCounterTestBase}. + */ +public class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTestBase { + + private static final ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + @AfterClass + public static void tearDown() throws Exception { + ZooKeeper.shutdown(); + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + } + + /** + * Tests that counter node is removed from ZooKeeper after shutdown. + */ + @Test + public void testShutdownRemovesState() throws Exception { + CheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + CuratorFramework client = ZooKeeper.getClient(); + assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); + + counter.shutdown(JobStatus.FINISHED); + assertNull(client.checkExists().forPath("/checkpoint-id-counter")); + } + + /** + * Tests that counter node is NOT removed from ZooKeeper after suspend. + */ + @Test + public void testSuspendKeepsState() throws Exception { + CheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + CuratorFramework client = ZooKeeper.getClient(); + assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); + + counter.shutdown(JobStatus.SUSPENDED); + assertNotNull(client.checkExists().forPath("/checkpoint-id-counter")); + } + + @Override + protected CheckpointIDCounter createCheckpointIdCounter() throws Exception { + return new ZooKeeperCheckpointIDCounter( + ZooKeeper.getClient(), + "/checkpoint-id-counter", + new DefaultLastStateConnectionStateListener()); + } +}
