KAFKA-12171: Migrate streams:test-utils module to JUnit 5 (#9856)

* replace `org.junit.Assert` by `org.junit.jupiter.api.Assertions`
* replace `org.junit` by `org.junit.jupiter.api`
* replace `org.junit.runners.Parameterized` by `org.junit.jupiter.params.ParameterizedTest`
* replace `org.junit.runners.Parameterized.Parameters` by `org.junit.jupiter.params.provider.{Arguments, MethodSource}`
* replace `Before` by `BeforeEach`
* replace `After` by `AfterEach`

Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Chia-Ping Tsai 2021-01-14 13:23:48 +08:00 committed by GitHub
parent 52b8aa0fdc
commit 04827dad51
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 142 additions and 132 deletions

View File

@ -241,7 +241,7 @@ subprojects {
}
}
def shouldUseJUnit5 = ["clients", "examples", "log4j-appender", "raft", "tools"].contains(it.project.name)
def shouldUseJUnit5 = ["clients", "examples", "log4j-appender", "raft", "test-utils", "tools"].contains(it.project.name)
def testLoggingEvents = ["passed", "skipped", "failed"]
def testShowStandardStreams = false
@ -1490,8 +1490,7 @@ project(':streams:test-utils') {
compile project(':clients')
testCompile project(':clients').sourceSets.test.output
testCompile libs.junitJupiterApi
testCompile libs.junitVintageEngine
testCompile libs.junitJupiter
testCompile libs.easymock
testCompile libs.hamcrest

View File

@ -30,18 +30,18 @@ import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.time.Duration;
import java.util.Iterator;
import java.util.Properties;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@SuppressWarnings("deprecation") // this is a test of a deprecated API
public class MockProcessorContextTest {

View File

@ -16,10 +16,10 @@
*/
package org.apache.kafka.streams;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class MockTimeTest {

View File

@ -28,9 +28,9 @@ import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.test.TestRecord;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -50,7 +50,7 @@ import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.hasProperty;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class TestTopicsTest {
private static final Logger log = LoggerFactory.getLogger(TestTopicsTest.class);
@ -66,7 +66,7 @@ public class TestTopicsTest {
private final Instant testBaseTime = Instant.parse("2019-06-01T10:00:00Z");
@Before
@BeforeEach
public void setup() {
final StreamsBuilder builder = new StreamsBuilder();
//Create Actual Stream Processing pipeline
@ -77,7 +77,7 @@ public class TestTopicsTest {
testDriver = new TopologyTestDriver(builder.build());
}
@After
@AfterEach
public void tearDown() {
try {
testDriver.close();
@ -326,14 +326,14 @@ public class TestTopicsTest {
public void testNonExistingOutputTopic() {
final TestOutputTopic<Long, String> outputTopic =
testDriver.createOutputTopic("no-exist", longSerde.deserializer(), stringSerde.deserializer());
assertThrows("Uninitialized topic", NoSuchElementException.class, outputTopic::readRecord);
assertThrows(NoSuchElementException.class, outputTopic::readRecord, "Uninitialized topic");
}
@Test
public void testNonUsedOutputTopic() {
final TestOutputTopic<Long, String> outputTopic =
testDriver.createOutputTopic(OUTPUT_TOPIC, longSerde.deserializer(), stringSerde.deserializer());
assertThrows("Uninitialized topic", NoSuchElementException.class, outputTopic::readRecord);
assertThrows(NoSuchElementException.class, outputTopic::readRecord, "Uninitialized topic");
}
@Test
@ -346,14 +346,14 @@ public class TestTopicsTest {
inputTopic.pipeInput("Hello");
assertThat(outputTopic.readValue(), equalTo("Hello"));
//No more output in topic
assertThrows("Empty topic", NoSuchElementException.class, outputTopic::readRecord);
assertThrows(NoSuchElementException.class, outputTopic::readRecord, "Empty topic");
}
@Test
public void testNonExistingInputTopic() {
final TestInputTopic<Long, String> inputTopic =
testDriver.createInputTopic("no-exist", longSerde.serializer(), stringSerde.serializer());
assertThrows("Unknown topic", IllegalArgumentException.class, () -> inputTopic.pipeInput(1L, "Hello"));
assertThrows(IllegalArgumentException.class, () -> inputTopic.pipeInput(1L, "Hello"), "Unknown topic");
}
@Test

View File

@ -0,0 +1,26 @@
/*
* 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.kafka.streams;
import java.util.Collections;
public class TopologyTestDriverAtLeastOnceTest extends TopologyTestDriverTest {
TopologyTestDriverAtLeastOnceTest() {
super(Collections.singletonMap(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.AT_LEAST_ONCE));
}
}

View File

@ -0,0 +1,26 @@
/*
* 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.kafka.streams;
import java.util.Collections;
public class TopologyTestDriverEosTest extends TopologyTestDriverTest {
TopologyTestDriverEosTest() {
super(Collections.singletonMap(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE));
}
}

View File

@ -53,11 +53,8 @@ import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.streams.test.TestRecord;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.time.Duration;
@ -87,16 +84,24 @@ import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.notNullValue;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
public abstract class TopologyTestDriverTest {
TopologyTestDriverTest(final Map<String, String> overrides) {
config = mkProperties(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "test-TopologyTestDriver"),
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath())
));
config.putAll(overrides);
}
@RunWith(value = Parameterized.class)
public class TopologyTestDriverTest {
private final static String SOURCE_TOPIC_1 = "source-topic-1";
private final static String SOURCE_TOPIC_2 = "source-topic-2";
private final static String SINK_TOPIC_1 = "sink-topic-1";
@ -131,25 +136,6 @@ public class TopologyTestDriverTest {
private final StringDeserializer stringDeserializer = new StringDeserializer();
private final LongDeserializer longDeserializer = new LongDeserializer();
@Parameterized.Parameters(name = "Eos enabled = {0}")
public static Collection<Object[]> data() {
final List<Object[]> values = new ArrayList<>();
for (final boolean eosEnabled : Arrays.asList(true, false)) {
values.add(new Object[] {eosEnabled});
}
return values;
}
public TopologyTestDriverTest(final boolean eosEnabled) {
config = mkProperties(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "test-TopologyTestDriver"),
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath())
));
if (eosEnabled) {
config.setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);
}
}
private final static class TTDTestRecord {
private final Object key;
private final Object value;
@ -319,7 +305,7 @@ public class TopologyTestDriverTest {
}
}
@After
@AfterEach
public void tearDown() {
if (testDriver != null) {
testDriver.close();
@ -462,10 +448,9 @@ public class TopologyTestDriverTest {
@Test
public void shouldCloseProcessor() {
testDriver = new TopologyTestDriver(setupSingleProcessorTopology(), config);
testDriver.close();
assertTrue(mockProcessors.get(0).closed);
// As testDriver is already closed, bypassing @After tearDown testDriver.close().
// As testDriver is already closed, bypassing @AfterEach tearDown testDriver.close().
testDriver = null;
}
@ -912,8 +897,8 @@ public class TopologyTestDriverTest {
testDriver = new TopologyTestDriver(setupGlobalStoreTopology(SOURCE_TOPIC_1));
final KeyValueStore<byte[], byte[]> globalStore = testDriver.getKeyValueStore(SOURCE_TOPIC_1 + "-globalStore");
Assert.assertNotNull(globalStore);
Assert.assertNotNull(testDriver.getAllStateStores().get(SOURCE_TOPIC_1 + "-globalStore"));
assertNotNull(globalStore);
assertNotNull(testDriver.getAllStateStores().get(SOURCE_TOPIC_1 + "-globalStore"));
pipeRecord(SOURCE_TOPIC_1, testRecord1);
@ -1562,15 +1547,13 @@ public class TopologyTestDriverTest {
assertNull(testDriver.getKeyValueStore("storeProcessorStore").get("a"));
testDriver.pipeRecord("input-topic", new TestRecord<>("a", 1L),
new StringSerializer(), new LongSerializer(), Instant.now());
Assert.assertEquals(1L, testDriver.getKeyValueStore("storeProcessorStore").get("a"));
assertEquals(1L, testDriver.getKeyValueStore("storeProcessorStore").get("a"));
}
try (final TopologyTestDriver testDriver = new TopologyTestDriver(topology, config)) {
assertNull(
"Closing the prior test driver should have cleaned up this store and value.",
testDriver.getKeyValueStore("storeProcessorStore").get("a")
);
assertNull(testDriver.getKeyValueStore("storeProcessorStore").get("a"),
"Closing the prior test driver should have cleaned up this store and value.");
}
}
@ -1583,8 +1566,8 @@ public class TopologyTestDriverTest {
Materialized.as("globalStore"));
try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build(), config)) {
final KeyValueStore<String, String> globalStore = testDriver.getKeyValueStore("globalStore");
Assert.assertNotNull(globalStore);
Assert.assertNotNull(testDriver.getAllStateStores().get("globalStore"));
assertNotNull(globalStore);
assertNotNull(testDriver.getAllStateStores().get("globalStore"));
testDriver.pipeRecord(
"topic",
new TestRecord<>("k1", "value1"),
@ -1592,7 +1575,7 @@ public class TopologyTestDriverTest {
new StringSerializer(),
Instant.now());
// we expect to have both in the global store, the one from pipeInput and the one from the producer
Assert.assertEquals("value1", globalStore.get("k1"));
assertEquals("value1", globalStore.get("k1"));
}
}

View File

@ -21,14 +21,11 @@ import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.TimestampedKeyValueStore;
import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.easymock.EasyMockRunner;
import org.easymock.Mock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.easymock.EasyMock;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static java.util.Arrays.asList;
import static org.easymock.EasyMock.expect;
@ -38,18 +35,14 @@ import static org.easymock.EasyMock.replay;
import static org.easymock.EasyMock.verify;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@RunWith(EasyMockRunner.class)
public class KeyValueStoreFacadeTest {
@Mock
private TimestampedKeyValueStore<String, String> mockedKeyValueTimestampStore;
@Mock
private KeyValueIterator<String, ValueAndTimestamp<String>> mockedKeyValueTimestampIterator;
private final TimestampedKeyValueStore<String, String> mockedKeyValueTimestampStore = EasyMock.mock(TimestampedKeyValueStore.class);
private KeyValueStoreFacade<String, String> keyValueStoreFacade;
@Before
@BeforeEach
public void setup() {
keyValueStoreFacade = new KeyValueStoreFacade<>(mockedKeyValueTimestampStore);
}

View File

@ -22,11 +22,9 @@ import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.TimestampedWindowStore;
import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.easymock.EasyMockRunner;
import org.easymock.Mock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.easymock.EasyMock;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.expectLastCall;
@ -36,14 +34,12 @@ import static org.easymock.EasyMock.verify;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
@RunWith(EasyMockRunner.class)
public class WindowStoreFacadeTest {
@Mock
private TimestampedWindowStore<String, String> mockedWindowTimestampStore;
private final TimestampedWindowStore<String, String> mockedWindowTimestampStore = EasyMock.mock(TimestampedWindowStore.class);
private WindowStoreFacade<String, String> windowStoreFacade;
@Before
@BeforeEach
public void setup() {
windowStoreFacade = new WindowStoreFacade<>(mockedWindowTimestampStore);
}

View File

@ -21,15 +21,15 @@ import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.KeyValue;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
@Deprecated
public class ConsumerRecordFactoryTest {

View File

@ -30,7 +30,7 @@ import org.apache.kafka.streams.processor.api.RecordMetadata;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.time.Duration;

View File

@ -31,36 +31,30 @@ import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
import org.apache.kafka.streams.state.WindowStore;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.common.utils.Utils.mkProperties;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertThrows;
@RunWith(value = Parameterized.class)
public class MockProcessorContextStateStoreTest {
private final StoreBuilder<StateStore> builder;
private final boolean timestamped;
private final boolean caching;
private final boolean logging;
@Parameterized.Parameters(name = "builder = {0}, timestamped = {1}, caching = {2}, logging = {3}")
public static Collection<Object[]> data() {
public static Stream<Arguments> parameters() {
final List<Boolean> booleans = asList(true, false);
final List<Object[]> values = new ArrayList<>();
final List<Arguments> values = new ArrayList<>();
for (final Boolean timestamped : booleans) {
for (final Boolean caching : booleans) {
@ -88,7 +82,7 @@ public class MockProcessorContextStateStoreTest {
builder.withLoggingDisabled();
}
values.add(new Object[] {builder, timestamped, caching, logging});
values.add(Arguments.of(builder, timestamped, caching, logging));
}
}
}
@ -121,7 +115,7 @@ public class MockProcessorContextStateStoreTest {
builder.withLoggingDisabled();
}
values.add(new Object[] {builder, timestamped, caching, logging});
values.add(Arguments.of(builder, timestamped, caching, logging));
}
}
}
@ -148,27 +142,20 @@ public class MockProcessorContextStateStoreTest {
builder.withLoggingDisabled();
}
values.add(new Object[] {builder, false, caching, logging});
values.add(Arguments.of(builder, false, caching, logging));
}
}
}
return values;
return values.stream();
}
public MockProcessorContextStateStoreTest(final StoreBuilder<StateStore> builder,
final boolean timestamped,
final boolean caching,
final boolean logging) {
this.builder = builder;
this.timestamped = timestamped;
this.caching = caching;
this.logging = logging;
}
@Test
public void shouldEitherInitOrThrow() {
@ParameterizedTest(name = "builder = {0}, timestamped = {1}, caching = {2}, logging = {3}")
@MethodSource(value = "parameters")
public void shouldEitherInitOrThrow(final StoreBuilder<StateStore> builder,
final boolean timestamped,
final boolean caching,
final boolean logging) {
final File stateDir = TestUtils.tempDirectory();
try {
final MockProcessorContext<Void, Void> context = new MockProcessorContext<>(

View File

@ -18,9 +18,9 @@ package org.apache.kafka.streams.test;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertThrows;
@Deprecated
public class OutputVerifierTest {

View File

@ -23,7 +23,7 @@ import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.record.TimestampType;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import java.time.Instant;
@ -31,9 +31,9 @@ import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.hasProperty;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class TestRecordTest {
private final String key = "testKey";

View File

@ -26,7 +26,7 @@ import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.WindowStore;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.io.IOException;