Browse Source

KAFKA-15659: Fix shouldInvokeUserDefinedGlobalStateRestoreListener flaky test (#14608)

Trying to fix flakiness for the shouldInvokeUserDefinedGlobalStateRestoreListener test introduced in #14519.

Fixes are:

-Do not use static membership.
-Always close the 2nd KafkaStreams instance.
-Await for the Kafka Streams instance to transition to a RUNNING state before proceeding.
-Added logging for the StateRestoreListener implementation.
-Reduce restore consumer MAX_POLL_RECORDS.

Reviewers: Anna Sophie Blee-Goldman <sophie@responsive.dev>
pull/14593/merge
Levani Kokhreidze 11 months ago committed by GitHub
parent
commit
986c1b1f31
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 50
      streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java

50
streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java

@ -70,6 +70,8 @@ import org.junit.jupiter.api.TestInfo; @@ -70,6 +70,8 @@ import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
@ -108,6 +110,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @@ -108,6 +110,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(600)
@Tag("integration")
public class RestoreIntegrationTest {
private static final Logger log = LoggerFactory.getLogger(RestoreIntegrationTest.class);
private static final Duration RESTORATION_DELAY = Duration.ofSeconds(1);
private static final int NUM_BROKERS = 1;
@ -523,13 +527,13 @@ public class RestoreIntegrationTest { @@ -523,13 +527,13 @@ public class RestoreIntegrationTest {
final Map<String, Object> kafkaStreams1Configuration = mkMap(
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"),
mkEntry(StreamsConfig.consumerPrefix(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG), appId + "-ks1"),
mkEntry(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 10)
mkEntry(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"),
mkEntry(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 5)
);
final Map<String, Object> kafkaStreams2Configuration = mkMap(
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"),
mkEntry(StreamsConfig.consumerPrefix(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG), appId + "-ks2"),
mkEntry(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 10)
mkEntry(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"),
mkEntry(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 5)
);
final StreamsBuilder builder = new StreamsBuilder();
@ -553,7 +557,7 @@ public class RestoreIntegrationTest { @@ -553,7 +557,7 @@ public class RestoreIntegrationTest {
kafkaStreams.close(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT));
IntegrationTestUtils.purgeLocalStreamsState(streamsConfigurations);
final TestStateRestoreListener kafkaStreams1StateRestoreListener = new TestStateRestoreListener(RESTORATION_DELAY);
final TestStateRestoreListener kafkaStreams1StateRestoreListener = new TestStateRestoreListener("ks1", RESTORATION_DELAY);
kafkaStreams = startKafkaStreams(builder, kafkaStreams1StateRestoreListener, kafkaStreams1Configuration);
assertTrue(kafkaStreams1StateRestoreListener.awaitUntilRestorationStarts());
@ -562,17 +566,23 @@ public class RestoreIntegrationTest { @@ -562,17 +566,23 @@ public class RestoreIntegrationTest {
// Simulate a new instance joining in the middle of the restoration.
// When this happens, some of the partitions that kafkaStreams1 was restoring will be migrated to kafkaStreams2,
// and kafkaStreams1 must call StateRestoreListener#onRestoreSuspended.
final TestStateRestoreListener kafkaStreams2StateRestoreListener = new TestStateRestoreListener(RESTORATION_DELAY);
final KafkaStreams kafkaStreams2 = startKafkaStreams(builder, kafkaStreams2StateRestoreListener, kafkaStreams2Configuration);
assertTrue(kafkaStreams1StateRestoreListener.awaitUntilRestorationSuspends());
final TestStateRestoreListener kafkaStreams2StateRestoreListener = new TestStateRestoreListener("ks2", RESTORATION_DELAY);
try (final KafkaStreams kafkaStreams2 = startKafkaStreams(builder,
kafkaStreams2StateRestoreListener,
kafkaStreams2Configuration)) {
waitForCondition(() -> State.RUNNING == kafkaStreams2.state(),
IntegrationTestUtils.DEFAULT_TIMEOUT,
() -> "kafkaStreams2 never transitioned to a RUNNING state.");
assertTrue(kafkaStreams2StateRestoreListener.awaitUntilRestorationStarts());
assertTrue(kafkaStreams1StateRestoreListener.awaitUntilRestorationSuspends());
assertTrue(kafkaStreams1StateRestoreListener.awaitUntilRestorationEnds());
assertTrue(kafkaStreams2StateRestoreListener.awaitUntilRestorationEnds());
assertTrue(kafkaStreams2StateRestoreListener.awaitUntilRestorationStarts());
// Cleanup
kafkaStreams2.close(Duration.ofMillis(IntegrationTestUtils.DEFAULT_TIMEOUT));
assertTrue(kafkaStreams1StateRestoreListener.awaitUntilRestorationEnds());
assertTrue(kafkaStreams2StateRestoreListener.awaitUntilRestorationEnds());
}
}
private void validateReceivedMessages(final List<KeyValue<Integer, Integer>> expectedRecords,
@ -601,8 +611,6 @@ public class RestoreIntegrationTest { @@ -601,8 +611,6 @@ public class RestoreIntegrationTest {
final StateRestoreListener stateRestoreListener,
final Map<String, Object> extraConfiguration) {
final Properties streamsConfiguration = props(mkObjectProperties(extraConfiguration));
streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 10);
final KafkaStreams kafkaStreams = new KafkaStreams(streamsBuilder.build(), streamsConfiguration);
kafkaStreams.setGlobalStateRestoreListener(stateRestoreListener);
@ -612,6 +620,7 @@ public class RestoreIntegrationTest { @@ -612,6 +620,7 @@ public class RestoreIntegrationTest {
}
private static final class TestStateRestoreListener implements StateRestoreListener {
private final String instanceName;
private final Duration onBatchRestoredSleepDuration;
private final CountDownLatch onRestoreStartLatch = new CountDownLatch(1);
@ -619,8 +628,9 @@ public class RestoreIntegrationTest { @@ -619,8 +628,9 @@ public class RestoreIntegrationTest {
private final CountDownLatch onRestoreSuspendedLatch = new CountDownLatch(1);
private final CountDownLatch onBatchRestoredLatch = new CountDownLatch(1);
TestStateRestoreListener(final Duration onBatchRestoredSleepDuration) {
TestStateRestoreListener(final String instanceName, final Duration onBatchRestoredSleepDuration) {
this.onBatchRestoredSleepDuration = onBatchRestoredSleepDuration;
this.instanceName = instanceName;
}
boolean awaitUntilRestorationStarts() throws InterruptedException {
@ -644,6 +654,8 @@ public class RestoreIntegrationTest { @@ -644,6 +654,8 @@ public class RestoreIntegrationTest {
final String storeName,
final long startingOffset,
final long endingOffset) {
log.info("[{}] called onRestoreStart. topicPartition={}, storeName={}, startingOffset={}, endingOffset={}",
instanceName, topicPartition, storeName, startingOffset, endingOffset);
onRestoreStartLatch.countDown();
}
@ -652,6 +664,8 @@ public class RestoreIntegrationTest { @@ -652,6 +664,8 @@ public class RestoreIntegrationTest {
final String storeName,
final long batchEndOffset,
final long numRestored) {
log.info("[{}] called onBatchRestored. topicPartition={}, storeName={}, batchEndOffset={}, numRestored={}",
instanceName, topicPartition, storeName, batchEndOffset, numRestored);
Utils.sleep(onBatchRestoredSleepDuration.toMillis());
onBatchRestoredLatch.countDown();
}
@ -660,6 +674,8 @@ public class RestoreIntegrationTest { @@ -660,6 +674,8 @@ public class RestoreIntegrationTest {
public void onRestoreEnd(final TopicPartition topicPartition,
final String storeName,
final long totalRestored) {
log.info("[{}] called onRestoreEnd. topicPartition={}, storeName={}, totalRestored={}",
instanceName, topicPartition, storeName, totalRestored);
onRestoreEndLatch.countDown();
}
@ -667,6 +683,8 @@ public class RestoreIntegrationTest { @@ -667,6 +683,8 @@ public class RestoreIntegrationTest {
public void onRestoreSuspended(final TopicPartition topicPartition,
final String storeName,
final long totalRestored) {
log.info("[{}] called onRestoreSuspended. topicPartition={}, storeName={}, totalRestored={}",
instanceName, topicPartition, storeName, totalRestored);
onRestoreSuspendedLatch.countDown();
}

Loading…
Cancel
Save