Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -847,7 +847,7 @@ private Set<KeyValue<Long, Long>> getMaxPerKey(final List<KeyValue<Long, Long>>
}

private void verifyStateStore(final KafkaStreams streams,
final Set<KeyValue<Long, Long>> expectedStoreContent) throws InterruptedException {
final Set<KeyValue<Long, Long>> expectedStoreContent) throws Exception {
final ReadOnlyKeyValueStore<Long, Long> store = IntegrationTestUtils
.getStore(300_000L, storeName, streams, QueryableStoreTypes.keyValueStore());
assertNotNull(store);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,10 +119,12 @@ public void before() throws Exception {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
streamsConfiguration.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 300);
globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()),
Materialized.<Long, String, KeyValueStore<Bytes, byte[]>>as(globalStore)
.withKeySerde(Serdes.Long())
.withValueSerde(Serdes.String()));
globalTable = builder.globalTable(
globalTableTopic,
Consumed.with(Serdes.Long(), Serdes.String()),
Materialized.<Long, String, KeyValueStore<Bytes, byte[]>>as(globalStore)
.withKeySerde(Serdes.Long())
.withValueSerde(Serdes.String()));
final Consumed<String, Long> stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());
stream = builder.stream(streamTopic, stringLongConsumed);
foreachAction = results::put;
Expand Down Expand Up @@ -154,7 +156,10 @@ public void shouldKStreamGlobalKTableLeftJoin() throws Exception {
TestUtils.waitForCondition(
() -> results.equals(expected),
30000L,
"waiting for initial values");
() -> "waiting for initial values;" +
"\n expected: " + expected +
"\n received: " + results
);


produceGlobalTableValues();
Expand All @@ -163,10 +168,27 @@ public void shouldKStreamGlobalKTableLeftJoin() throws Exception {
.getStore(globalStore, kafkaStreams, QueryableStoreTypes.keyValueStore());
assertNotNull(replicatedStore);


final Map<Long, String> expectedState = new HashMap<>();
expectedState.put(1L, "F");
expectedState.put(2L, "G");
expectedState.put(3L, "H");
expectedState.put(4L, "I");
expectedState.put(5L, "J");

final Map<Long, String> globalState = new HashMap<>();
TestUtils.waitForCondition(
() -> "J".equals(replicatedStore.get(5L)),
() -> {
globalState.clear();
replicatedStore.all().forEachRemaining(pair -> globalState.put(pair.key, pair.value));
return globalState.equals(expectedState);
},
30000,
"waiting for data in replicated store");
() -> "waiting for data in replicated store" +
"\n expected: " + expectedState +
"\n received: " + globalState
);


produceTopicValues(streamTopic);

Expand All @@ -179,7 +201,10 @@ public void shouldKStreamGlobalKTableLeftJoin() throws Exception {
TestUtils.waitForCondition(
() -> results.equals(expected),
30000L,
"waiting for final values");
() -> "waiting for final values" +
"\n expected: " + expected +
"\n received: " + results
);
}

@Test
Expand All @@ -199,7 +224,10 @@ public void shouldKStreamGlobalKTableJoin() throws Exception {
TestUtils.waitForCondition(
() -> results.equals(expected),
30000L,
"waiting for initial values");
() -> "waiting for initial values" +
"\n expected: " + expected +
"\n received: " + results
);


produceGlobalTableValues();
Expand All @@ -208,10 +236,27 @@ public void shouldKStreamGlobalKTableJoin() throws Exception {
.getStore(globalStore, kafkaStreams, QueryableStoreTypes.keyValueStore());
assertNotNull(replicatedStore);


final Map<Long, String> expectedState = new HashMap<>();
expectedState.put(1L, "F");
expectedState.put(2L, "G");
expectedState.put(3L, "H");
expectedState.put(4L, "I");
expectedState.put(5L, "J");

final Map<Long, String> globalState = new HashMap<>();
TestUtils.waitForCondition(
() -> "J".equals(replicatedStore.get(5L)),
() -> {
globalState.clear();
replicatedStore.all().forEachRemaining(pair -> globalState.put(pair.key, pair.value));
return globalState.equals(expectedState);
},
30000,
"waiting for data in replicated store");
() -> "waiting for data in replicated store" +
"\n expected: " + expectedState +
"\n received: " + globalState
);


produceTopicValues(streamTopic);

Expand All @@ -224,7 +269,10 @@ public void shouldKStreamGlobalKTableJoin() throws Exception {
TestUtils.waitForCondition(
() -> results.equals(expected),
30000L,
"waiting for final values");
() -> "waiting for final values" +
"\n expected: " + expected +
"\n received: " + results
);
}

@Test
Expand All @@ -243,9 +291,10 @@ public void shouldRestoreTransactionalMessages() throws Exception {
.getStore(globalStore, kafkaStreams, QueryableStoreTypes.keyValueStore());
assertNotNull(store);

final Map<Long, String> result = new HashMap<>();
TestUtils.waitForCondition(
() -> {
final Map<Long, String> result = new HashMap<>();
result.clear();
final Iterator<KeyValue<Long, String>> it = store.all();
while (it.hasNext()) {
final KeyValue<Long, String> kv = it.next();
Expand All @@ -254,7 +303,10 @@ public void shouldRestoreTransactionalMessages() throws Exception {
return result.equals(expected);
},
30000L,
"waiting for initial values");
() -> "waiting for initial values" +
"\n expected: " + expected +
"\n received: " + results
);
}

@Test
Expand All @@ -275,18 +327,18 @@ public void shouldNotRestoreAbortedMessages() throws Exception {
.getStore(globalStore, kafkaStreams, QueryableStoreTypes.keyValueStore());
assertNotNull(store);

final Map<Long, String> result = new HashMap<>();
TestUtils.waitForCondition(
() -> {
final Map<Long, String> result = new HashMap<>();
final Iterator<KeyValue<Long, String>> it = store.all();
while (it.hasNext()) {
final KeyValue<Long, String> kv = it.next();
result.put(kv.key, kv.value);
}
result.clear();
store.all().forEachRemaining(pair -> result.put(pair.key, pair.value));
return result.equals(expected);
},
30000L,
"waiting for initial values");
() -> "waiting for initial values" +
"\n expected: " + expected +
"\n received: " + results
);
}

private void createTopics() throws Exception {
Expand All @@ -303,76 +355,92 @@ private void startStreams() {
}

private void produceTopicValues(final String topic) {
final Properties config = new Properties();
config.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true");

IntegrationTestUtils.produceKeyValuesSynchronously(
topic,
Arrays.asList(
new KeyValue<>("a", 1L),
new KeyValue<>("b", 2L),
new KeyValue<>("c", 3L),
new KeyValue<>("d", 4L),
new KeyValue<>("e", 5L)),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
StringSerializer.class,
LongSerializer.class,
new Properties()),
mockTime);
topic,
Arrays.asList(
new KeyValue<>("a", 1L),
new KeyValue<>("b", 2L),
new KeyValue<>("c", 3L),
new KeyValue<>("d", 4L),
new KeyValue<>("e", 5L)
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
StringSerializer.class,
LongSerializer.class,
config
),
mockTime
);
}

private void produceAbortedMessages() throws Exception {
final Properties properties = new Properties();
properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "someid");
properties.put(ProducerConfig.RETRIES_CONFIG, 1);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

This might be the actually fix. Not sure why we set retries to one, but if we would loose input data, we would never complete the result and the test would time out. (Maybe not relevant for aborted message, but same below)


IntegrationTestUtils.produceAbortedKeyValuesSynchronouslyWithTimestamp(
globalTableTopic, Arrays.asList(
new KeyValue<>(1L, "A"),
new KeyValue<>(2L, "B"),
new KeyValue<>(3L, "C"),
new KeyValue<>(4L, "D")
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
properties),
mockTime.milliseconds());
globalTableTopic, Arrays.asList(
new KeyValue<>(1L, "A"),
new KeyValue<>(2L, "B"),
new KeyValue<>(3L, "C"),
new KeyValue<>(4L, "D")
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
properties
),
mockTime.milliseconds()
);
}

private void produceInitialGlobalTableValues() {
final Properties properties = new Properties();
properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "someid");
properties.put(ProducerConfig.RETRIES_CONFIG, 1);

IntegrationTestUtils.produceKeyValuesSynchronously(
globalTableTopic,
Arrays.asList(
new KeyValue<>(1L, "A"),
new KeyValue<>(2L, "B"),
new KeyValue<>(3L, "C"),
new KeyValue<>(4L, "D")
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
properties),
mockTime,
true);
globalTableTopic,
Arrays.asList(
new KeyValue<>(1L, "A"),
new KeyValue<>(2L, "B"),
new KeyValue<>(3L, "C"),
new KeyValue<>(4L, "D")
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
properties
),
mockTime,
true
);
}

private void produceGlobalTableValues() {
final Properties config = new Properties();
config.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true");

IntegrationTestUtils.produceKeyValuesSynchronously(
globalTableTopic,
Arrays.asList(
new KeyValue<>(1L, "F"),
new KeyValue<>(2L, "G"),
new KeyValue<>(3L, "H"),
new KeyValue<>(4L, "I"),
new KeyValue<>(5L, "J")),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
new Properties()),
mockTime);
globalTableTopic,
Arrays.asList(
new KeyValue<>(1L, "F"),
new KeyValue<>(2L, "G"),
new KeyValue<>(3L, "H"),
new KeyValue<>(4L, "I"),
new KeyValue<>(5L, "J")
),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
LongSerializer.class,
StringSerializer.class,
config
),
mockTime
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,8 @@ private void startStreams() {
private <K, V> void validateReceivedMessages(final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer,
final List<KeyValueTimestamp<K, V>> expectedRecords)
throws InterruptedException {
throws Exception {

final String safeTestName = safeUniqueTestName(getClass(), testName);
final Properties consumerProperties = new Properties();
consumerProperties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -808,14 +808,17 @@ private void startStreams() {
private <K, V> List<KeyValueTimestamp<K, V>> receiveMessages(final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer,
final int numMessages)
throws InterruptedException {
throws Exception {

return receiveMessages(keyDeserializer, valueDeserializer, null, numMessages);
}

private <K, V> List<KeyValueTimestamp<K, V>> receiveMessages(final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer,
final Class innerClass,
final int numMessages) throws InterruptedException {
final int numMessages)
throws Exception {

final String safeTestName = safeUniqueTestName(getClass(), testName);
final Properties consumerProperties = new Properties();
consumerProperties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
Expand All @@ -835,9 +838,9 @@ private <K, V> List<KeyValueTimestamp<K, V>> receiveMessages(final Deserializer<
}

private <K, V> List<KeyValueTimestamp<K, V>> receiveMessagesWithTimestamp(final Deserializer<K> keyDeserializer,
final Deserializer<V> valueDeserializer,
final Class innerClass,
final int numMessages) throws InterruptedException {
final Deserializer<V> valueDeserializer,
final Class innerClass,
final int numMessages) throws Exception {
final String safeTestName = safeUniqueTestName(getClass(), testName);
final Properties consumerProperties = new Properties();
consumerProperties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -807,7 +807,7 @@ private KafkaStreams startStreams(final StreamsBuilder builder,

private <K, V> void validateReceivedMessages(final Deserializer<K> keySerializer,
final Deserializer<V> valueSerializer,
final List<KeyValue<K, V>> expectedRecords) throws InterruptedException {
final List<KeyValue<K, V>> expectedRecords) throws Exception {

final String safeTestName = safeUniqueTestName(getClass(), testName);
final Properties consumerProperties = new Properties();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutionException;

@Category({IntegrationTest.class})
public class StateRestorationIntegrationTest {
Expand Down Expand Up @@ -77,7 +76,7 @@ public void setUp() throws Exception {
}

@Test
public void shouldRestoreNullRecord() throws InterruptedException, ExecutionException {
public void shouldRestoreNullRecord() throws Exception {
builder.table(INPUT_TOPIC, Materialized.<Integer, Bytes>as(
Stores.persistentTimestampedKeyValueStore(STATE_STORE_NAME))
.withKeySerde(Serdes.Integer())
Expand Down
Loading