MINOR Fix a few test names (#17788)

Remove or update custom display names to make sure we actually include the test method as the first part of the display name.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
This commit is contained in:
David Arthur 2024-11-13 13:28:38 -05:00 committed by GitHub
parent 6fc6e87382
commit 48ff6a6b53
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 30 additions and 43 deletions

View File

@ -103,7 +103,7 @@ public class RangeAssignorTest {
assertTrue(assignment.get(consumer1).isEmpty()); assertTrue(assignment.get(consumer1).isEmpty());
} }
@ParameterizedTest(name = "rackConfig = {0}") @ParameterizedTest(name = "{displayName}.rackConfig = {0}")
@EnumSource(RackConfig.class) @EnumSource(RackConfig.class)
public void testOneConsumerOneTopic(RackConfig rackConfig) { public void testOneConsumerOneTopic(RackConfig rackConfig) {
initializeRacks(rackConfig); initializeRacks(rackConfig);

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.List; import java.util.List;
@ -41,7 +40,6 @@ class SampledStatTest {
} }
@Test @Test
@DisplayName("Sample should be purged if doesn't overlap the window")
public void testSampleIsPurgedIfDoesntOverlap() { public void testSampleIsPurgedIfDoesntOverlap() {
MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2);
@ -50,11 +48,10 @@ class SampledStatTest {
time.sleep(2500); time.sleep(2500);
double numSamples = stat.measure(config, time.milliseconds()); double numSamples = stat.measure(config, time.milliseconds());
assertEquals(0, numSamples); assertEquals(0, numSamples, "Sample should be purged if doesn't overlap the window");
} }
@Test @Test
@DisplayName("Sample should be kept if overlaps the window")
public void testSampleIsKeptIfOverlaps() { public void testSampleIsKeptIfOverlaps() {
MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2);
@ -63,11 +60,10 @@ class SampledStatTest {
time.sleep(1500); time.sleep(1500);
double numSamples = stat.measure(config, time.milliseconds()); double numSamples = stat.measure(config, time.milliseconds());
assertEquals(1, numSamples); assertEquals(1, numSamples, "Sample should be kept if overlaps the window");
} }
@Test @Test
@DisplayName("Sample should be kept if overlaps the window and is n+1")
public void testSampleIsKeptIfOverlapsAndExtra() { public void testSampleIsKeptIfOverlapsAndExtra() {
MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2);
@ -80,7 +76,7 @@ class SampledStatTest {
stat.record(config, 1, time.milliseconds()); stat.record(config, 1, time.milliseconds());
double numSamples = stat.measure(config, time.milliseconds()); double numSamples = stat.measure(config, time.milliseconds());
assertEquals(3, numSamples); assertEquals(3, numSamples, "Sample should be kept if overlaps the window and is n+1");
} }
// Creates a sample with events at the start and at the end. Positions clock at the end. // Creates a sample with events at the start and at the end. Positions clock at the end.

View File

@ -76,7 +76,7 @@ public class SslVersionsTransportLayerTest {
* Tests that connection success with the default TLS version. * Tests that connection success with the default TLS version.
* Note that debug mode for javax.net.ssl can be enabled via {@code System.setProperty("javax.net.debug", "ssl:handshake");} * Note that debug mode for javax.net.ssl can be enabled via {@code System.setProperty("javax.net.debug", "ssl:handshake");}
*/ */
@ParameterizedTest(name = "tlsServerProtocol = {0}, tlsClientProtocol = {1}") @ParameterizedTest(name = "testTlsDefaults(tlsServerProtocol = {0}, tlsClientProtocol = {1})")
@MethodSource("parameters") @MethodSource("parameters")
public void testTlsDefaults(List<String> serverProtocols, List<String> clientProtocols) throws Exception { public void testTlsDefaults(List<String> serverProtocols, List<String> clientProtocols) throws Exception {
// Create certificates for use by client and server. Add server cert to client truststore and vice versa. // Create certificates for use by client and server. Add server cert to client truststore and vice versa.

View File

@ -86,7 +86,7 @@ public class LazyDownConversionRecordsTest {
* than the number of bytes we get after conversion. This causes overflow message batch(es) to be appended towards the * than the number of bytes we get after conversion. This causes overflow message batch(es) to be appended towards the
* end of the converted output. * end of the converted output.
*/ */
@ParameterizedTest(name = "compressionType={0}, toMagic={1}, overflow={2}") @ParameterizedTest
@MethodSource("parameters") @MethodSource("parameters")
public void testConversion(CompressionType compressionType, byte toMagic, boolean overflow) throws IOException { public void testConversion(CompressionType compressionType, byte toMagic, boolean overflow) throws IOException {
doTestConversion(compressionType, toMagic, overflow); doTestConversion(compressionType, toMagic, overflow);

View File

@ -69,7 +69,7 @@ public class RemoteLogMetadataCacheTest {
} }
} }
@ParameterizedTest(name = "isInitialized={0}") @ParameterizedTest
@ValueSource(booleans = {true, false}) @ValueSource(booleans = {true, false})
public void testCacheUpdateMetadataOnInvalidArgs(boolean isInitialized) { public void testCacheUpdateMetadataOnInvalidArgs(boolean isInitialized) {
if (isInitialized) { if (isInitialized) {

View File

@ -55,7 +55,6 @@ import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
@ -148,8 +147,8 @@ public class KafkaStreamsTelemetryIntegrationTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(strings = {"INFO", "DEBUG", "TRACE"}) @ValueSource(strings = {"INFO", "DEBUG", "TRACE"})
@DisplayName("End-to-end test validating metrics pushed to broker")
public void shouldPushMetricsToBroker(final String recordingLevel) throws Exception { public void shouldPushMetricsToBroker(final String recordingLevel) throws Exception {
// End-to-end test validating metrics pushed to broker
streamsApplicationProperties = props(true); streamsApplicationProperties = props(true);
streamsApplicationProperties.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel); streamsApplicationProperties.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel);
final Topology topology = simpleTopology(); final Topology topology = simpleTopology();
@ -202,8 +201,8 @@ public class KafkaStreamsTelemetryIntegrationTest {
@ParameterizedTest @ParameterizedTest
@MethodSource("singleAndMultiTaskParameters") @MethodSource("singleAndMultiTaskParameters")
@DisplayName("Streams metrics should get passed to Admin and Consumer")
public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws Exception { public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws Exception {
// Streams metrics should get passed to Admin and Consumer
streamsApplicationProperties = props(stateUpdaterEnabled); streamsApplicationProperties = props(stateUpdaterEnabled);
final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology();
@ -232,8 +231,8 @@ public class KafkaStreamsTelemetryIntegrationTest {
@ParameterizedTest @ParameterizedTest
@MethodSource("multiTaskParameters") @MethodSource("multiTaskParameters")
@DisplayName("Correct streams metrics should get passed with dynamic membership")
public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws Exception { public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws Exception {
// Correct streams metrics should get passed with dynamic membership
streamsApplicationProperties = props(stateUpdaterEnabled); streamsApplicationProperties = props(stateUpdaterEnabled);
streamsApplicationProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); streamsApplicationProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1");
streamsApplicationProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); streamsApplicationProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1");
@ -324,8 +323,8 @@ public class KafkaStreamsTelemetryIntegrationTest {
} }
@Test @Test
@DisplayName("Streams metrics should not be visible in client metrics")
public void passedMetricsShouldNotLeakIntoClientMetrics() throws Exception { public void passedMetricsShouldNotLeakIntoClientMetrics() throws Exception {
// Streams metrics should not be visible in client metrics
streamsApplicationProperties = props(true); streamsApplicationProperties = props(true);
final Topology topology = complexTopology(); final Topology topology = complexTopology();

View File

@ -26,7 +26,6 @@ import org.apache.kafka.common.utils.Time;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays; import java.util.Arrays;
@ -75,26 +74,25 @@ class StreamsClientMetricsDelegatingReporterTest {
} }
@Test @Test
@DisplayName("Should register metrics from init method")
public void shouldInitMetrics() { public void shouldInitMetrics() {
final List<KafkaMetric> metrics = Arrays.asList(streamClientMetricOne, streamClientMetricTwo, streamClientMetricThree, kafkaMetricWithThreadIdTag); final List<KafkaMetric> metrics = Arrays.asList(streamClientMetricOne, streamClientMetricTwo, streamClientMetricThree, kafkaMetricWithThreadIdTag);
streamsClientMetricsDelegatingReporter.init(metrics); streamsClientMetricsDelegatingReporter.init(metrics);
final List<KafkaMetric> expectedMetrics = Arrays.asList(streamClientMetricOne, streamClientMetricTwo, streamClientMetricThree); final List<KafkaMetric> expectedMetrics = Arrays.asList(streamClientMetricOne, streamClientMetricTwo, streamClientMetricThree);
assertEquals(expectedMetrics, mockAdminClient.addedMetrics()); assertEquals(expectedMetrics, mockAdminClient.addedMetrics(),
"Should register metrics from init method");
} }
@Test @Test
@DisplayName("Should register client instance metrics only")
public void shouldRegisterCorrectMetrics() { public void shouldRegisterCorrectMetrics() {
streamsClientMetricsDelegatingReporter.metricChange(kafkaMetricWithThreadIdTag); streamsClientMetricsDelegatingReporter.metricChange(kafkaMetricWithThreadIdTag);
assertEquals(0, mockAdminClient.addedMetrics().size()); assertEquals(0, mockAdminClient.addedMetrics().size());
streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricOne); streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricOne);
assertEquals(1, mockAdminClient.addedMetrics().size()); assertEquals(1, mockAdminClient.addedMetrics().size(),
"Should register client instance metrics only");
} }
@Test @Test
@DisplayName("Should remove client instance metrics")
public void metricRemoval() { public void metricRemoval() {
streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricOne); streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricOne);
streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricTwo); streamsClientMetricsDelegatingReporter.metricChange(streamClientMetricTwo);
@ -102,6 +100,7 @@ class StreamsClientMetricsDelegatingReporterTest {
assertEquals(3, mockAdminClient.addedMetrics().size()); assertEquals(3, mockAdminClient.addedMetrics().size());
streamsClientMetricsDelegatingReporter.metricRemoval(streamClientMetricOne); streamsClientMetricsDelegatingReporter.metricRemoval(streamClientMetricOne);
assertEquals(2, mockAdminClient.addedMetrics().size()); assertEquals(2, mockAdminClient.addedMetrics().size(),
"Should remove client instance metrics");
} }
} }

View File

@ -27,7 +27,6 @@ import org.apache.kafka.common.utils.Time;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays; import java.util.Arrays;
@ -84,23 +83,22 @@ class StreamsThreadMetricsDelegatingReporterTest {
@Test @Test
@DisplayName("Init method should register metrics it receives as parameters")
public void shouldInitMetrics() { public void shouldInitMetrics() {
final List<KafkaMetric> allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); final List<KafkaMetric> allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag);
final List<KafkaMetric> expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); final List<KafkaMetric> expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag);
streamsThreadMetricsDelegatingReporter.init(allMetrics); streamsThreadMetricsDelegatingReporter.init(allMetrics);
assertEquals(expectedMetrics, mockConsumer.addedMetrics()); assertEquals(expectedMetrics, mockConsumer.addedMetrics(),
"Init method should register metrics it receives as parameters");
} }
@Test @Test
@DisplayName("Should register metrics with thread-id in tag map")
public void shouldRegisterMetrics() { public void shouldRegisterMetrics() {
streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricOneHasThreadIdTag); streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricOneHasThreadIdTag);
assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0),
"Should register metrics with thread-id in tag map");
} }
@Test @Test
@DisplayName("Should remove metrics")
public void shouldRemoveMetrics() { public void shouldRemoveMetrics() {
streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricOneHasThreadIdTag); streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricOneHasThreadIdTag);
streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricTwoHasThreadIdTag); streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricTwoHasThreadIdTag);
@ -109,13 +107,14 @@ class StreamsThreadMetricsDelegatingReporterTest {
assertEquals(expected, mockConsumer.addedMetrics()); assertEquals(expected, mockConsumer.addedMetrics());
streamsThreadMetricsDelegatingReporter.metricRemoval(kafkaMetricOneHasThreadIdTag); streamsThreadMetricsDelegatingReporter.metricRemoval(kafkaMetricOneHasThreadIdTag);
expected = Arrays.asList(kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); expected = Arrays.asList(kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag);
assertEquals(expected, mockConsumer.addedMetrics()); assertEquals(expected, mockConsumer.addedMetrics(),
"Should remove metrics");
} }
@Test @Test
@DisplayName("Should not register metrics without thread-id tag")
public void shouldNotRegisterMetricsWithoutThreadIdTag() { public void shouldNotRegisterMetricsWithoutThreadIdTag() {
streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricWithoutThreadIdTag); streamsThreadMetricsDelegatingReporter.metricChange(kafkaMetricWithoutThreadIdTag);
assertEquals(0, mockConsumer.addedMetrics().size()); assertEquals(0, mockConsumer.addedMetrics().size(),
"Should not register metrics without thread-id tag");
} }
} }

View File

@ -34,7 +34,6 @@ import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.ArrayList; import java.util.ArrayList;
@ -50,13 +49,11 @@ import static java.util.Arrays.asList;
public class KStreamNewProcessorApiTest { public class KStreamNewProcessorApiTest {
@Test @Test
@DisplayName("Should attach the state store using ConnectedStoreProvider")
void shouldGetStateStoreWithConnectedStoreProvider() { void shouldGetStateStoreWithConnectedStoreProvider() {
runTest(false); runTest(false);
} }
@Test @Test
@DisplayName("Should attach the state store StreamBuilder.addStateStore")
void shouldGetStateStoreWithStreamBuilder() { void shouldGetStateStoreWithStreamBuilder() {
runTest(true); runTest(true);
} }

View File

@ -19,7 +19,6 @@ package org.apache.kafka.streams.query;
import org.apache.kafka.streams.query.internals.SucceededQueryResult; import org.apache.kafka.streams.query.internals.SucceededQueryResult;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -40,26 +39,24 @@ class StateQueryResultTest {
} }
@Test @Test
@DisplayName("Zero query results shouldn't error")
void getOnlyPartitionResultNoResultsTest() { void getOnlyPartitionResultNoResultsTest() {
stringStateQueryResult.addResult(0, noResultsFound); stringStateQueryResult.addResult(0, noResultsFound);
final QueryResult<String> result = stringStateQueryResult.getOnlyPartitionResult(); final QueryResult<String> result = stringStateQueryResult.getOnlyPartitionResult();
assertThat(result, nullValue()); assertThat("Zero query results shouldn't error", result, nullValue());
} }
@Test @Test
@DisplayName("Valid query results still works")
void getOnlyPartitionResultWithSingleResultTest() { void getOnlyPartitionResultWithSingleResultTest() {
stringStateQueryResult.addResult(0, validResult); stringStateQueryResult.addResult(0, validResult);
final QueryResult<String> result = stringStateQueryResult.getOnlyPartitionResult(); final QueryResult<String> result = stringStateQueryResult.getOnlyPartitionResult();
assertThat(result.getResult(), is("Foo")); assertThat("Valid query results still works", result.getResult(), is("Foo"));
} }
@Test @Test
@DisplayName("More than one query result throws IllegalArgumentException ")
void getOnlyPartitionResultMultipleResults() { void getOnlyPartitionResultMultipleResults() {
stringStateQueryResult.addResult(0, validResult); stringStateQueryResult.addResult(0, validResult);
stringStateQueryResult.addResult(1, validResult); stringStateQueryResult.addResult(1, validResult);
assertThrows(IllegalArgumentException.class, () -> stringStateQueryResult.getOnlyPartitionResult()); assertThrows(IllegalArgumentException.class, () -> stringStateQueryResult.getOnlyPartitionResult(),
"More than one query result throws IllegalArgumentException");
} }
} }

View File

@ -155,7 +155,7 @@ public class MockProcessorContextStateStoreTest {
return values.stream(); return values.stream();
} }
@ParameterizedTest(name = "builder = {0}, timestamped = {1}, caching = {2}, logging = {3}") @ParameterizedTest
@MethodSource(value = "parameters") @MethodSource(value = "parameters")
public void shouldEitherInitOrThrow(final StoreBuilder<StateStore> builder, public void shouldEitherInitOrThrow(final StoreBuilder<StateStore> builder,
final boolean timestamped, final boolean timestamped,