Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MINOR: Further reduce runtime for metrics integration tests #8514

Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Bytes;
Expand Down Expand Up @@ -264,66 +263,46 @@ private void startApplication() throws InterruptedException {

private void produceRecordsForTwoSegments(final Duration segmentInterval) throws Exception {
final MockTime mockTime = new MockTime(Math.max(segmentInterval.toMillis(), 60_000L));
final Properties props = TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties());
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "A")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()),
props,
mockTime.milliseconds()
);
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "B")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()),
props,
mockTime.milliseconds()
);
}

private void produceRecordsForClosingWindow(final Duration windowSize) throws Exception {
final MockTime mockTime = new MockTime(windowSize.toMillis() + 1);
final Properties props = TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties());
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "A")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()),
props,
mockTime.milliseconds()
);
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "B")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()),
props,
mockTime.milliseconds()
);
}

private void waitUntilAllRecordsAreConsumed(final int numberOfExpectedRecords) throws Exception {
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(
TestUtils.consumerConfig(
CLUSTER.bootstrapServers(),
"consumerApp",
LongDeserializer.class,
LongDeserializer.class,
new Properties()
),
STREAM_OUTPUT_1,
numberOfExpectedRecords
);
}

private void closeApplication() throws Exception {
kafkaStreams.close();
kafkaStreams.cleanUp();
Expand Down Expand Up @@ -423,8 +402,6 @@ private void shouldAddMetricsForWindowStoreAndSuppressionBuffer(final String bui

verifyStateMetric(State.RUNNING);

waitUntilAllRecordsAreConsumed(1);

checkWindowStoreAndSuppressionBufferMetrics(builtInMetricsVersion);

closeApplication();
Expand Down Expand Up @@ -465,8 +442,6 @@ private void shouldAddMetricsForSessionStore(final String builtInMetricsVersion)

verifyStateMetric(State.RUNNING);

waitUntilAllRecordsAreConsumed(2);

checkSessionStoreMetrics(builtInMetricsVersion);

closeApplication();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,12 @@

import org.apache.kafka.common.Metric;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.IntegerDeserializer;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
Expand Down Expand Up @@ -73,8 +71,10 @@ public class RocksDBMetricsIntegrationTest {
@ClassRule
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);

private static final String STREAM_INPUT = "STREAM_INPUT";
private static final String STREAM_OUTPUT = "STREAM_OUTPUT";
private static final String STREAM_INPUT_ONE = "STREAM_INPUT_ONE";
private static final String STREAM_OUTPUT_ONE = "STREAM_OUTPUT_ONE";
private static final String STREAM_INPUT_TWO = "STREAM_INPUT_TWO";
private static final String STREAM_OUTPUT_TWO = "STREAM_OUTPUT_TWO";
private static final String MY_STORE_PERSISTENT_KEY_VALUE = "myStorePersistentKeyValue";
private static final Duration WINDOW_SIZE = Duration.ofMillis(50);
private static final long TIMEOUT = 60000;
Expand Down Expand Up @@ -112,12 +112,13 @@ public static Collection<Object[]> data() {

@Before
public void before() throws Exception {
CLUSTER.createTopic(STREAM_INPUT, 1, 3);
CLUSTER.createTopic(STREAM_INPUT_ONE, 1, 3);
CLUSTER.createTopic(STREAM_INPUT_TWO, 1, 3);
}

@After
public void after() throws Exception {
CLUSTER.deleteTopicsAndWait(STREAM_INPUT, STREAM_OUTPUT);
CLUSTER.deleteTopicsAndWait(STREAM_INPUT_ONE, STREAM_INPUT_TWO, STREAM_OUTPUT_ONE, STREAM_OUTPUT_TWO);
}

@FunctionalInterface
Expand All @@ -126,58 +127,23 @@ private interface MetricsVerifier {
}

@Test
public void shouldExposeRocksDBMetricsForNonSegmentedStateStoreBeforeAndAfterFailureWithEmptyStateDir() throws Exception {
public void shouldExposeRocksDBMetricsBeforeAndAfterFailureWithEmptyStateDir() throws Exception {
final Properties streamsConfiguration = streamsConfig();
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
final StreamsBuilder builder = builderForNonSegmentedStateStore();
final String metricsScope = "rocksdb-state-id";
final StreamsBuilder builder = builderForStateStores();

cleanUpStateRunVerifyAndClose(
builder,
streamsConfiguration,
IntegerDeserializer.class,
StringDeserializer.class,
this::verifyThatRocksDBMetricsAreExposed,
metricsScope
this::verifyThatRocksDBMetricsAreExposed
);

// simulated failure

cleanUpStateRunVerifyAndClose(
builder,
streamsConfiguration,
IntegerDeserializer.class,
StringDeserializer.class,
this::verifyThatRocksDBMetricsAreExposed,
metricsScope
);
}

@Test
public void shouldExposeRocksDBMetricsForSegmentedStateStoreBeforeAndAfterFailureWithEmptyStateDir() throws Exception {
final Properties streamsConfiguration = streamsConfig();
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
final StreamsBuilder builder = builderForSegmentedStateStore();
final String metricsScope = "rocksdb-window-state-id";

cleanUpStateRunVerifyAndClose(
builder,
streamsConfiguration,
LongDeserializer.class,
LongDeserializer.class,
this::verifyThatRocksDBMetricsAreExposed,
metricsScope
);

// simulated failure

cleanUpStateRunVerifyAndClose(
builder,
streamsConfiguration,
LongDeserializer.class,
LongDeserializer.class,
this::verifyThatRocksDBMetricsAreExposed,
metricsScope
this::verifyThatRocksDBMetricsAreExposed
);
}

Expand All @@ -193,18 +159,14 @@ private Properties streamsConfig() {
return streamsConfiguration;
}

private StreamsBuilder builderForNonSegmentedStateStore() {
private StreamsBuilder builderForStateStores() {
final StreamsBuilder builder = new StreamsBuilder();
// create two state stores, one non-segmented and one segmented
builder.table(
STREAM_INPUT,
STREAM_INPUT_ONE,
Materialized.as(Stores.persistentKeyValueStore(MY_STORE_PERSISTENT_KEY_VALUE)).withCachingEnabled()
).toStream().to(STREAM_OUTPUT);
return builder;
}

private StreamsBuilder builderForSegmentedStateStore() {
final StreamsBuilder builder = new StreamsBuilder();
builder.stream(STREAM_INPUT, Consumed.with(Serdes.Integer(), Serdes.String()))
).toStream().to(STREAM_OUTPUT_ONE);
builder.stream(STREAM_INPUT_TWO, Consumed.with(Serdes.Integer(), Serdes.String()))
.groupByKey()
.windowedBy(TimeWindows.of(WINDOW_SIZE).grace(Duration.ZERO))
.aggregate(() -> 0L,
Expand All @@ -214,70 +176,55 @@ private StreamsBuilder builderForSegmentedStateStore() {
.withRetention(WINDOW_SIZE))
.toStream()
.map((key, value) -> KeyValue.pair(value, value))
.to(STREAM_OUTPUT, Produced.with(Serdes.Long(), Serdes.Long()));
.to(STREAM_OUTPUT_TWO, Produced.with(Serdes.Long(), Serdes.Long()));
return builder;
}

private void cleanUpStateRunVerifyAndClose(final StreamsBuilder builder,
final Properties streamsConfiguration,
final Class outputKeyDeserializer,
final Class outputValueDeserializer,
final MetricsVerifier metricsVerifier,
final String metricsScope) throws Exception {
final MetricsVerifier metricsVerifier) throws Exception {
final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
kafkaStreams.cleanUp();
produceRecords();

StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams, TIMEOUT);

IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(
TestUtils.consumerConfig(
CLUSTER.bootstrapServers(),
"consumerApp",
outputKeyDeserializer,
outputValueDeserializer,
new Properties()
),
STREAM_OUTPUT,
1
);
metricsVerifier.verify(kafkaStreams, metricsScope);
metricsVerifier.verify(kafkaStreams, "rocksdb-state-id");
metricsVerifier.verify(kafkaStreams, "rocksdb-window-state-id");
kafkaStreams.close();
}

private void produceRecords() throws Exception {
final MockTime mockTime = new MockTime(WINDOW_SIZE.toMillis());
final Properties prop = TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()
);
// non-segmented store do not need records with different timestamps
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "A")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()
),
STREAM_INPUT_ONE,
Utils.mkSet(new KeyValue<>(1, "A"), new KeyValue<>(1, "B"), new KeyValue<>(1, "C")),
prop,
mockTime.milliseconds()
);
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "B")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()
),
STREAM_INPUT_TWO,
Collections.singleton(new KeyValue<>(1, "A")),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we write into STREAM_INPUT_TWO with 3 calls instead of just one call passing in all 3 records at once?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note we use a mocktime with auto tick WINDOW_SIZE.toMillis(): and each time its milliseconds() is called it would auto-advance, we need the produced records with those advanced timestamps.

prop,
mockTime.milliseconds()
);
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT,
Collections.singletonList(new KeyValue<>(1, "C")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
IntegerSerializer.class,
StringSerializer.class,
new Properties()
),
STREAM_INPUT_TWO,
Collections.singleton(new KeyValue<>(1, "B")),
prop,
mockTime.milliseconds()
);
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
STREAM_INPUT_TWO,
Collections.singleton(new KeyValue<>(1, "C")),
prop,
mockTime.milliseconds()
);
}
Expand Down Expand Up @@ -319,18 +266,6 @@ private void checkMetricByName(final List<Metric> listMetric,
}
}

private void verifyThatBytesWrittenTotalIncreases(final KafkaStreams kafkaStreams,
final String metricsScope) throws InterruptedException {
final List<Metric> metric = getRocksDBMetrics(kafkaStreams, metricsScope).stream()
.filter(m -> BYTES_WRITTEN_TOTAL.equals(m.metricName().name()))
.collect(Collectors.toList());
TestUtils.waitForCondition(
() -> (double) metric.get(0).metricValue() > 0,
TIMEOUT,
() -> "RocksDB metric bytes.written.total did not increase in " + TIMEOUT + " ms"
);
}

Copy link
Contributor

@cadonna cadonna Apr 20, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you! I forgot to delete this in my refactoring.

private List<Metric> getRocksDBMetrics(final KafkaStreams kafkaStreams,
final String metricsScope) {
return new ArrayList<Metric>(kafkaStreams.metrics().values()).stream()
Expand Down