|
|
|
@ -16,12 +16,14 @@
@@ -16,12 +16,14 @@
|
|
|
|
|
*/ |
|
|
|
|
package org.apache.kafka.streams.integration; |
|
|
|
|
|
|
|
|
|
import kafka.tools.ConsoleConsumer; |
|
|
|
|
import kafka.utils.MockTime; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerConfig; |
|
|
|
|
import org.apache.kafka.common.serialization.Deserializer; |
|
|
|
|
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.Serde; |
|
|
|
|
import org.apache.kafka.common.serialization.Serdes; |
|
|
|
|
import org.apache.kafka.common.serialization.StringDeserializer; |
|
|
|
|
import org.apache.kafka.common.serialization.StringSerializer; |
|
|
|
@ -43,10 +45,14 @@ import org.apache.kafka.streams.kstream.Materialized;
@@ -43,10 +45,14 @@ import org.apache.kafka.streams.kstream.Materialized;
|
|
|
|
|
import org.apache.kafka.streams.kstream.Produced; |
|
|
|
|
import org.apache.kafka.streams.kstream.Reducer; |
|
|
|
|
import org.apache.kafka.streams.kstream.Serialized; |
|
|
|
|
import org.apache.kafka.streams.kstream.SessionWindowedDeserializer; |
|
|
|
|
import org.apache.kafka.streams.kstream.SessionWindows; |
|
|
|
|
import org.apache.kafka.streams.kstream.TimeWindowedDeserializer; |
|
|
|
|
import org.apache.kafka.streams.kstream.TimeWindows; |
|
|
|
|
import org.apache.kafka.streams.kstream.Windowed; |
|
|
|
|
import org.apache.kafka.streams.kstream.WindowedSerdes; |
|
|
|
|
import org.apache.kafka.streams.kstream.internals.SessionWindow; |
|
|
|
|
import org.apache.kafka.streams.kstream.internals.TimeWindow; |
|
|
|
|
import org.apache.kafka.streams.state.KeyValueIterator; |
|
|
|
|
import org.apache.kafka.streams.state.QueryableStoreTypes; |
|
|
|
|
import org.apache.kafka.streams.state.ReadOnlySessionStore; |
|
|
|
@ -60,14 +66,18 @@ import org.junit.ClassRule;
@@ -60,14 +66,18 @@ import org.junit.ClassRule;
|
|
|
|
|
import org.junit.Test; |
|
|
|
|
import org.junit.experimental.categories.Category; |
|
|
|
|
|
|
|
|
|
import java.io.ByteArrayOutputStream; |
|
|
|
|
import java.io.IOException; |
|
|
|
|
import java.io.PrintStream; |
|
|
|
|
import java.util.Arrays; |
|
|
|
|
import java.util.Collections; |
|
|
|
|
import java.util.Comparator; |
|
|
|
|
import java.util.HashMap; |
|
|
|
|
import java.util.HashSet; |
|
|
|
|
import java.util.List; |
|
|
|
|
import java.util.Map; |
|
|
|
|
import java.util.Properties; |
|
|
|
|
import java.util.Set; |
|
|
|
|
import java.util.concurrent.CountDownLatch; |
|
|
|
|
import java.util.concurrent.TimeUnit; |
|
|
|
|
|
|
|
|
@ -75,6 +85,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
@@ -75,6 +85,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
|
|
|
|
|
import static org.hamcrest.MatcherAssert.assertThat; |
|
|
|
|
import static org.hamcrest.core.Is.is; |
|
|
|
|
import static org.junit.Assert.assertFalse; |
|
|
|
|
import static org.junit.Assert.assertTrue; |
|
|
|
|
|
|
|
|
|
@Category({IntegrationTest.class}) |
|
|
|
|
public class KStreamAggregationIntegrationTest { |
|
|
|
@ -205,32 +216,36 @@ public class KStreamAggregationIntegrationTest {
@@ -205,32 +216,36 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
produceMessages(secondBatchTimestamp); |
|
|
|
|
produceMessages(secondBatchTimestamp); |
|
|
|
|
|
|
|
|
|
Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); |
|
|
|
|
groupedStream |
|
|
|
|
.windowedBy(TimeWindows.of(500L)) |
|
|
|
|
.reduce(reducer) |
|
|
|
|
.toStream(new KeyValueMapper<Windowed<String>, String, String>() { |
|
|
|
|
@Override |
|
|
|
|
public String apply(final Windowed<String> windowedKey, final String value) { |
|
|
|
|
return windowedKey.key() + "@" + windowedKey.window().start(); |
|
|
|
|
} |
|
|
|
|
}) |
|
|
|
|
.to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); |
|
|
|
|
|
|
|
|
|
.toStream() |
|
|
|
|
.to(outputTopic, Produced.with(windowedSerde, Serdes.String())); |
|
|
|
|
|
|
|
|
|
startStreams(); |
|
|
|
|
|
|
|
|
|
final List<KeyValue<String, String>> windowedOutput = receiveMessages( |
|
|
|
|
new StringDeserializer(), |
|
|
|
|
final List<KeyValue<Windowed<String>, String>> windowedOutput = receiveMessages( |
|
|
|
|
new TimeWindowedDeserializer<String>(), |
|
|
|
|
new StringDeserializer(), |
|
|
|
|
String.class, |
|
|
|
|
15); |
|
|
|
|
|
|
|
|
|
final Comparator<KeyValue<String, String>> |
|
|
|
|
// read from ConsoleConsumer
|
|
|
|
|
String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer( |
|
|
|
|
new TimeWindowedDeserializer<String>(), |
|
|
|
|
new StringDeserializer(), |
|
|
|
|
String.class, |
|
|
|
|
15); |
|
|
|
|
|
|
|
|
|
final Comparator<KeyValue<Windowed<String>, String>> |
|
|
|
|
comparator = |
|
|
|
|
new Comparator<KeyValue<String, String>>() { |
|
|
|
|
new Comparator<KeyValue<Windowed<String>, String>>() { |
|
|
|
|
@Override |
|
|
|
|
public int compare(final KeyValue<String, String> o1, |
|
|
|
|
final KeyValue<String, String> o2) { |
|
|
|
|
return KStreamAggregationIntegrationTest.compare(o1, o2); |
|
|
|
|
public int compare(final KeyValue<Windowed<String>, String> o1, |
|
|
|
|
final KeyValue<Windowed<String>, String> o2) { |
|
|
|
|
final int keyComparison = o1.key.key().compareTo(o2.key.key()); |
|
|
|
|
return keyComparison == 0 ? o1.value.compareTo(o2.value) : keyComparison; |
|
|
|
|
} |
|
|
|
|
}; |
|
|
|
|
|
|
|
|
@ -238,25 +253,36 @@ public class KStreamAggregationIntegrationTest {
@@ -238,25 +253,36 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
final long firstBatchWindow = firstBatchTimestamp / 500 * 500; |
|
|
|
|
final long secondBatchWindow = secondBatchTimestamp / 500 * 500; |
|
|
|
|
|
|
|
|
|
assertThat(windowedOutput, is( |
|
|
|
|
Arrays.asList( |
|
|
|
|
new KeyValue<>("A@" + firstBatchWindow, "A"), |
|
|
|
|
new KeyValue<>("A@" + secondBatchWindow, "A"), |
|
|
|
|
new KeyValue<>("A@" + secondBatchWindow, "A:A"), |
|
|
|
|
new KeyValue<>("B@" + firstBatchWindow, "B"), |
|
|
|
|
new KeyValue<>("B@" + secondBatchWindow, "B"), |
|
|
|
|
new KeyValue<>("B@" + secondBatchWindow, "B:B"), |
|
|
|
|
new KeyValue<>("C@" + firstBatchWindow, "C"), |
|
|
|
|
new KeyValue<>("C@" + secondBatchWindow, "C"), |
|
|
|
|
new KeyValue<>("C@" + secondBatchWindow, "C:C"), |
|
|
|
|
new KeyValue<>("D@" + firstBatchWindow, "D"), |
|
|
|
|
new KeyValue<>("D@" + secondBatchWindow, "D"), |
|
|
|
|
new KeyValue<>("D@" + secondBatchWindow, "D:D"), |
|
|
|
|
new KeyValue<>("E@" + firstBatchWindow, "E"), |
|
|
|
|
new KeyValue<>("E@" + secondBatchWindow, "E"), |
|
|
|
|
new KeyValue<>("E@" + secondBatchWindow, "E:E") |
|
|
|
|
) |
|
|
|
|
)); |
|
|
|
|
List<KeyValue<Windowed<String>, String>> expectResult = Arrays.asList( |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "A"), |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A"), |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A:A"), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "B"), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "B"), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "B:B"), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "C"), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "C"), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "C:C"), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "D"), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "D"), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "D:D"), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "E"), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "E"), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "E:E") |
|
|
|
|
); |
|
|
|
|
assertThat(windowedOutput, is(expectResult)); |
|
|
|
|
|
|
|
|
|
Set<String> expectResultString = new HashSet<>(expectResult.size()); |
|
|
|
|
for (KeyValue<Windowed<String>, String> eachRecord: expectResult) { |
|
|
|
|
expectResultString.add(eachRecord.toString()); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// check every message is contained in the expect result
|
|
|
|
|
String[] allRecords = resultFromConsoleConsumer.split("\n"); |
|
|
|
|
for (String record: allRecords) { |
|
|
|
|
record = "KeyValue(" + record + ")"; |
|
|
|
|
assertTrue(expectResultString.contains(record)); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@SuppressWarnings("deprecation") |
|
|
|
@ -309,34 +335,39 @@ public class KStreamAggregationIntegrationTest {
@@ -309,34 +335,39 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
produceMessages(secondTimestamp); |
|
|
|
|
produceMessages(secondTimestamp); |
|
|
|
|
|
|
|
|
|
Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); |
|
|
|
|
groupedStream.windowedBy(TimeWindows.of(500L)) |
|
|
|
|
.aggregate( |
|
|
|
|
initializer, |
|
|
|
|
aggregator, |
|
|
|
|
Materialized.<String, Integer, WindowStore<Bytes, byte[]>>with(null, Serdes.Integer()) |
|
|
|
|
) |
|
|
|
|
.toStream(new KeyValueMapper<Windowed<String>, Integer, String>() { |
|
|
|
|
@Override |
|
|
|
|
public String apply(final Windowed<String> windowedKey, final Integer value) { |
|
|
|
|
return windowedKey.key() + "@" + windowedKey.window().start(); |
|
|
|
|
} |
|
|
|
|
}) |
|
|
|
|
.to(outputTopic, Produced.with(Serdes.String(), Serdes.Integer())); |
|
|
|
|
.toStream() |
|
|
|
|
.to(outputTopic, Produced.with(windowedSerde, Serdes.Integer())); |
|
|
|
|
|
|
|
|
|
startStreams(); |
|
|
|
|
|
|
|
|
|
final List<KeyValue<String, Integer>> windowedMessages = receiveMessages( |
|
|
|
|
new StringDeserializer(), |
|
|
|
|
final List<KeyValue<Windowed<String>, Integer>> windowedMessages = receiveMessages( |
|
|
|
|
new TimeWindowedDeserializer<String>(), |
|
|
|
|
new IntegerDeserializer(), |
|
|
|
|
String.class, |
|
|
|
|
15); |
|
|
|
|
|
|
|
|
|
final Comparator<KeyValue<String, Integer>> |
|
|
|
|
// read from ConsoleConsumer
|
|
|
|
|
String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer( |
|
|
|
|
new TimeWindowedDeserializer<String>(), |
|
|
|
|
new IntegerDeserializer(), |
|
|
|
|
String.class, |
|
|
|
|
15); |
|
|
|
|
|
|
|
|
|
final Comparator<KeyValue<Windowed<String>, Integer>> |
|
|
|
|
comparator = |
|
|
|
|
new Comparator<KeyValue<String, Integer>>() { |
|
|
|
|
new Comparator<KeyValue<Windowed<String>, Integer>>() { |
|
|
|
|
@Override |
|
|
|
|
public int compare(final KeyValue<String, Integer> o1, |
|
|
|
|
final KeyValue<String, Integer> o2) { |
|
|
|
|
return KStreamAggregationIntegrationTest.compare(o1, o2); |
|
|
|
|
public int compare(final KeyValue<Windowed<String>, Integer> o1, |
|
|
|
|
final KeyValue<Windowed<String>, Integer> o2) { |
|
|
|
|
final int keyComparison = o1.key.key().compareTo(o2.key.key()); |
|
|
|
|
return keyComparison == 0 ? o1.value.compareTo(o2.value) : keyComparison; |
|
|
|
|
} |
|
|
|
|
}; |
|
|
|
|
|
|
|
|
@ -345,24 +376,37 @@ public class KStreamAggregationIntegrationTest {
@@ -345,24 +376,37 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
final long firstWindow = firstTimestamp / 500 * 500; |
|
|
|
|
final long secondWindow = secondTimestamp / 500 * 500; |
|
|
|
|
|
|
|
|
|
assertThat(windowedMessages, is( |
|
|
|
|
Arrays.asList( |
|
|
|
|
new KeyValue<>("A@" + firstWindow, 1), |
|
|
|
|
new KeyValue<>("A@" + secondWindow, 1), |
|
|
|
|
new KeyValue<>("A@" + secondWindow, 2), |
|
|
|
|
new KeyValue<>("B@" + firstWindow, 1), |
|
|
|
|
new KeyValue<>("B@" + secondWindow, 1), |
|
|
|
|
new KeyValue<>("B@" + secondWindow, 2), |
|
|
|
|
new KeyValue<>("C@" + firstWindow, 1), |
|
|
|
|
new KeyValue<>("C@" + secondWindow, 1), |
|
|
|
|
new KeyValue<>("C@" + secondWindow, 2), |
|
|
|
|
new KeyValue<>("D@" + firstWindow, 1), |
|
|
|
|
new KeyValue<>("D@" + secondWindow, 1), |
|
|
|
|
new KeyValue<>("D@" + secondWindow, 2), |
|
|
|
|
new KeyValue<>("E@" + firstWindow, 1), |
|
|
|
|
new KeyValue<>("E@" + secondWindow, 1), |
|
|
|
|
new KeyValue<>("E@" + secondWindow, 2) |
|
|
|
|
))); |
|
|
|
|
List<KeyValue<Windowed<String>, Integer>> expectResult = Arrays.asList( |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1), |
|
|
|
|
new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2)); |
|
|
|
|
|
|
|
|
|
assertThat(windowedMessages, is(expectResult)); |
|
|
|
|
|
|
|
|
|
Set<String> expectResultString = new HashSet<>(expectResult.size()); |
|
|
|
|
for (KeyValue<Windowed<String>, Integer> eachRecord: expectResult) { |
|
|
|
|
expectResultString.add(eachRecord.toString()); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// check every message is contained in the expect result
|
|
|
|
|
String[] allRecords = resultFromConsoleConsumer.split("\n"); |
|
|
|
|
for (String record: allRecords) { |
|
|
|
|
record = "KeyValue(" + record + ")"; |
|
|
|
|
assertTrue(expectResultString.contains(record)); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private void shouldCountHelper() throws Exception { |
|
|
|
@ -685,26 +729,66 @@ public class KStreamAggregationIntegrationTest {
@@ -685,26 +729,66 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
kafkaStreams.start(); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private <K, V> List<KeyValue<K, V>> receiveMessages(final Deserializer<K> |
|
|
|
|
keyDeserializer, |
|
|
|
|
final Deserializer<V> |
|
|
|
|
valueDeserializer, |
|
|
|
|
final int numMessages) |
|
|
|
|
throws InterruptedException { |
|
|
|
|
return receiveMessages(keyDeserializer, valueDeserializer, null, numMessages); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private <K, V> List<KeyValue<K, V>> receiveMessages(final Deserializer<K> |
|
|
|
|
keyDeserializer, |
|
|
|
|
final Deserializer<V> |
|
|
|
|
valueDeserializer, |
|
|
|
|
final Class innerClass, |
|
|
|
|
final int numMessages) throws InterruptedException { |
|
|
|
|
final Properties consumerProperties = new Properties(); |
|
|
|
|
consumerProperties |
|
|
|
|
.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); |
|
|
|
|
consumerProperties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); |
|
|
|
|
consumerProperties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "kgroupedstream-test-" + testNo); |
|
|
|
|
consumerProperties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); |
|
|
|
|
consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); |
|
|
|
|
consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); |
|
|
|
|
if (keyDeserializer instanceof TimeWindowedDeserializer || keyDeserializer instanceof SessionWindowedDeserializer) { |
|
|
|
|
consumerProperties.setProperty(StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS, |
|
|
|
|
Serdes.serdeFrom(innerClass).getClass().getName()); |
|
|
|
|
} |
|
|
|
|
return IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( |
|
|
|
|
consumerProperties, |
|
|
|
|
outputTopic, |
|
|
|
|
numMessages, |
|
|
|
|
60 * 1000); |
|
|
|
|
|
|
|
|
|
consumerProperties, |
|
|
|
|
outputTopic, |
|
|
|
|
numMessages, |
|
|
|
|
60 * 1000); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private <K, V> String readWindowedKeyedMessagesViaConsoleConsumer(final Deserializer<K> keyDeserializer, |
|
|
|
|
final Deserializer<V> valueDeserializer, |
|
|
|
|
final Class innerClass, |
|
|
|
|
final int numMessages) { |
|
|
|
|
ByteArrayOutputStream newConsole = new ByteArrayOutputStream(); |
|
|
|
|
PrintStream originalStream = System.out; |
|
|
|
|
try (PrintStream newStream = new PrintStream(newConsole)) { |
|
|
|
|
System.setOut(newStream); |
|
|
|
|
|
|
|
|
|
String keySeparator = ", "; |
|
|
|
|
// manually construct the console consumer argument array
|
|
|
|
|
String[] args = new String[] { |
|
|
|
|
"--bootstrap-server", CLUSTER.bootstrapServers(), |
|
|
|
|
"--from-beginning", |
|
|
|
|
"--property", "print.key=true", |
|
|
|
|
"--topic", outputTopic, |
|
|
|
|
"--max-messages", String.valueOf(numMessages), |
|
|
|
|
"--property", "key.deserializer=" + keyDeserializer.getClass().getName(), |
|
|
|
|
"--property", "value.deserializer=" + valueDeserializer.getClass().getName(), |
|
|
|
|
"--property", "key.separator=" + keySeparator, |
|
|
|
|
"--" + StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS, Serdes.serdeFrom(innerClass).getClass().getName() |
|
|
|
|
}; |
|
|
|
|
|
|
|
|
|
ConsoleConsumer.messageCount_$eq(0); //reset the message count
|
|
|
|
|
ConsoleConsumer.run(new ConsoleConsumer.ConsumerConfig(args)); |
|
|
|
|
newStream.flush(); |
|
|
|
|
System.setOut(originalStream); |
|
|
|
|
return newConsole.toString(); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|