|
|
|
@ -207,7 +207,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
produceMessages(secondBatchTimestamp);
|
|
|
|
|
produceMessages(secondBatchTimestamp);
|
|
|
|
|
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class, 500L);
|
|
|
|
|
groupedStream
|
|
|
|
|
.windowedBy(TimeWindows.of(ofMillis(500L)))
|
|
|
|
|
.reduce(reducer)
|
|
|
|
@ -235,25 +235,27 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
.thenComparing(KeyValueTimestamp::value);
|
|
|
|
|
|
|
|
|
|
windowedOutput.sort(comparator);
|
|
|
|
|
final long firstBatchWindow = firstBatchTimestamp / 500 * 500;
|
|
|
|
|
final long secondBatchWindow = secondBatchTimestamp / 500 * 500;
|
|
|
|
|
final long firstBatchWindowStart = firstBatchTimestamp / 500 * 500;
|
|
|
|
|
final long firstBatchWindowEnd = firstBatchWindowStart + 500;
|
|
|
|
|
final long secondBatchWindowStart = secondBatchTimestamp / 500 * 500;
|
|
|
|
|
final long secondBatchWindowEnd = secondBatchWindowStart + 500;
|
|
|
|
|
|
|
|
|
|
final List<KeyValueTimestamp<Windowed<String>, String>> expectResult = Arrays.asList(
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "A", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A:A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "B", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "B:B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "C", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "C:C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "D", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "D:D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "E", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "E:E", secondBatchTimestamp)
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchWindowStart, firstBatchWindowEnd)), "A", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "A:A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchWindowStart, firstBatchWindowEnd)), "B", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "B:B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchWindowStart, firstBatchWindowEnd)), "C", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "C:C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchWindowStart, firstBatchWindowEnd)), "D", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "D:D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchWindowStart, firstBatchWindowEnd)), "E", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchWindowStart, secondBatchWindowEnd)), "E:E", secondBatchTimestamp)
|
|
|
|
|
);
|
|
|
|
|
assertThat(windowedOutput, is(expectResult));
|
|
|
|
|
|
|
|
|
@ -314,7 +316,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
produceMessages(secondTimestamp);
|
|
|
|
|
produceMessages(secondTimestamp);
|
|
|
|
|
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class, 500L);
|
|
|
|
|
groupedStream.windowedBy(TimeWindows.of(ofMillis(500L)))
|
|
|
|
|
.aggregate(
|
|
|
|
|
initializer,
|
|
|
|
@ -327,7 +329,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
startStreams();
|
|
|
|
|
|
|
|
|
|
final List<KeyValueTimestamp<Windowed<String>, Integer>> windowedMessages = receiveMessagesWithTimestamp(
|
|
|
|
|
new TimeWindowedDeserializer<>(),
|
|
|
|
|
new TimeWindowedDeserializer<>(new StringDeserializer(), 500L),
|
|
|
|
|
new IntegerDeserializer(),
|
|
|
|
|
String.class,
|
|
|
|
|
15);
|
|
|
|
@ -345,25 +347,27 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
.thenComparingInt(KeyValueTimestamp::value);
|
|
|
|
|
windowedMessages.sort(comparator);
|
|
|
|
|
|
|
|
|
|
final long firstWindow = firstTimestamp / 500 * 500;
|
|
|
|
|
final long secondWindow = secondTimestamp / 500 * 500;
|
|
|
|
|
final long firstWindowStart = firstTimestamp / 500 * 500;
|
|
|
|
|
final long firstWindowEnd = firstWindowStart + 500;
|
|
|
|
|
final long secondWindowStart = secondTimestamp / 500 * 500;
|
|
|
|
|
final long secondWindowEnd = secondWindowStart + 500;
|
|
|
|
|
|
|
|
|
|
final List<KeyValueTimestamp<Windowed<String>, Integer>> expectResult = Arrays.asList(
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2, secondTimestamp));
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstWindowStart, firstWindowEnd)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondWindowStart, secondWindowEnd)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondWindowStart, secondWindowEnd)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstWindowStart, firstWindowEnd)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondWindowStart, secondWindowEnd)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondWindowStart, secondWindowEnd)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstWindowStart, firstWindowEnd)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondWindowStart, secondWindowEnd)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondWindowStart, secondWindowEnd)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstWindowStart, firstWindowEnd)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondWindowStart, secondWindowEnd)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondWindowStart, secondWindowEnd)), 2, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstWindowStart, firstWindowEnd)), 1, firstTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondWindowStart, secondWindowEnd)), 1, secondTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondWindowStart, secondWindowEnd)), 2, secondTimestamp));
|
|
|
|
|
|
|
|
|
|
assertThat(windowedMessages, is(expectResult));
|
|
|
|
|
|
|
|
|
@ -471,7 +475,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
final long thirdBatchTimestamp = firstBatchTimestamp + timeDifference - 100L;
|
|
|
|
|
produceMessages(thirdBatchTimestamp);
|
|
|
|
|
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class, timeDifference);
|
|
|
|
|
groupedStream
|
|
|
|
|
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(timeDifference), ofMillis(2000L)))
|
|
|
|
|
.reduce(reducer)
|
|
|
|
@ -499,52 +503,56 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
|
|
|
|
|
windowedOutput.sort(comparator);
|
|
|
|
|
final long firstBatchLeftWindowStart = firstBatchTimestamp - timeDifference;
|
|
|
|
|
final long firstBatchLeftWindowEnd = firstBatchLeftWindowStart + timeDifference;
|
|
|
|
|
final long firstBatchRightWindowStart = firstBatchTimestamp + 1;
|
|
|
|
|
final long firstBatchRightWindowEnd = firstBatchRightWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final long secondBatchLeftWindowStart = secondBatchTimestamp - timeDifference;
|
|
|
|
|
final long secondBatchLeftWindowEnd = secondBatchLeftWindowStart + timeDifference;
|
|
|
|
|
final long secondBatchRightWindowStart = secondBatchTimestamp + 1;
|
|
|
|
|
final long secondBatchRightWindowEnd = secondBatchRightWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final long thirdBatchLeftWindowStart = thirdBatchTimestamp - timeDifference;
|
|
|
|
|
|
|
|
|
|
final long thirdBatchLeftWindowEnd = thirdBatchLeftWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final List<KeyValueTimestamp<Windowed<String>, String>> expectResult = Arrays.asList(
|
|
|
|
|
// A @ firstBatchTimestamp left window created when A @ firstBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchLeftWindowStart, Long.MAX_VALUE)), "A", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), "A", firstBatchTimestamp),
|
|
|
|
|
// A @ firstBatchTimestamp right window created when A @ secondBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "A", secondBatchTimestamp),
|
|
|
|
|
// A @ secondBatchTimestamp right window created when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchRightWindowStart, Long.MAX_VALUE)), "A", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), "A", thirdBatchTimestamp),
|
|
|
|
|
// A @ secondBatchTimestamp left window created when A @ secondBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchLeftWindowStart, Long.MAX_VALUE)), "A:A", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), "A:A", secondBatchTimestamp),
|
|
|
|
|
// A @ firstBatchTimestamp right window updated when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "A:A", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "A:A", thirdBatchTimestamp),
|
|
|
|
|
// A @ thirdBatchTimestamp left window created when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(thirdBatchLeftWindowStart, Long.MAX_VALUE)), "A:A:A", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), "A:A:A", thirdBatchTimestamp),
|
|
|
|
|
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchLeftWindowStart, Long.MAX_VALUE)), "B", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchRightWindowStart, Long.MAX_VALUE)), "B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchLeftWindowStart, Long.MAX_VALUE)), "B:B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "B:B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(thirdBatchLeftWindowStart, Long.MAX_VALUE)), "B:B:B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchLeftWindowStart, Long.MAX_VALUE)), "C", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchRightWindowStart, Long.MAX_VALUE)), "C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchLeftWindowStart, Long.MAX_VALUE)), "C:C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "C:C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(thirdBatchLeftWindowStart, Long.MAX_VALUE)), "C:C:C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchLeftWindowStart, Long.MAX_VALUE)), "D", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchRightWindowStart, Long.MAX_VALUE)), "D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchLeftWindowStart, Long.MAX_VALUE)), "D:D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "D:D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(thirdBatchLeftWindowStart, Long.MAX_VALUE)), "D:D:D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchLeftWindowStart, Long.MAX_VALUE)), "E", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchRightWindowStart, Long.MAX_VALUE)), "E", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchLeftWindowStart, Long.MAX_VALUE)), "E:E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, Long.MAX_VALUE)), "E:E", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(thirdBatchLeftWindowStart, Long.MAX_VALUE)), "E:E:E", thirdBatchTimestamp)
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), "B", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), "B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), "B:B", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "B:B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), "B:B:B", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), "C", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), "C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), "C:C", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "C:C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), "C:C:C", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), "D", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), "D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), "D:D", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "D:D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), "D:D:D", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), "E", firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), "E", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), "E:E", secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), "E:E", thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), "E:E:E", thirdBatchTimestamp)
|
|
|
|
|
);
|
|
|
|
|
assertThat(windowedOutput, is(expectResult));
|
|
|
|
|
|
|
|
|
@ -571,7 +579,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
final long thirdBatchTimestamp = firstBatchTimestamp + timeDifference - 100L;
|
|
|
|
|
produceMessages(thirdBatchTimestamp);
|
|
|
|
|
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
|
|
|
|
|
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class, timeDifference);
|
|
|
|
|
groupedStream.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(500L), ofMinutes(5)))
|
|
|
|
|
.aggregate(
|
|
|
|
|
initializer,
|
|
|
|
@ -602,50 +610,57 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
.thenComparingInt(KeyValueTimestamp::value);
|
|
|
|
|
windowedMessages.sort(comparator);
|
|
|
|
|
|
|
|
|
|
final long firstBatchLeftWindow = firstBatchTimestamp - timeDifference;
|
|
|
|
|
final long firstBatchRightWindow = firstBatchTimestamp + 1;
|
|
|
|
|
final long secondBatchLeftWindow = secondBatchTimestamp - timeDifference;
|
|
|
|
|
final long secondBatchRightWindow = secondBatchTimestamp + 1;
|
|
|
|
|
final long thirdBatchLeftWindow = thirdBatchTimestamp - timeDifference;
|
|
|
|
|
final long firstBatchLeftWindowStart = firstBatchTimestamp - timeDifference;
|
|
|
|
|
final long firstBatchLeftWindowEnd = firstBatchLeftWindowStart + timeDifference;
|
|
|
|
|
final long firstBatchRightWindowStart = firstBatchTimestamp + 1;
|
|
|
|
|
final long firstBatchRightWindowEnd = firstBatchRightWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final long secondBatchLeftWindowStart = secondBatchTimestamp - timeDifference;
|
|
|
|
|
final long secondBatchLeftWindowEnd = secondBatchLeftWindowStart + timeDifference;
|
|
|
|
|
final long secondBatchRightWindowStart = secondBatchTimestamp + 1;
|
|
|
|
|
final long secondBatchRightWindowEnd = secondBatchRightWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final long thirdBatchLeftWindowStart = thirdBatchTimestamp - timeDifference;
|
|
|
|
|
final long thirdBatchLeftWindowEnd = thirdBatchLeftWindowStart + timeDifference;
|
|
|
|
|
|
|
|
|
|
final List<KeyValueTimestamp<Windowed<String>, Integer>> expectResult = Arrays.asList(
|
|
|
|
|
// A @ firstBatchTimestamp left window created when A @ firstBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchLeftWindow, Long.MAX_VALUE)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), 1, firstBatchTimestamp),
|
|
|
|
|
// A @ firstBatchTimestamp right window created when A @ secondBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 1, secondBatchTimestamp),
|
|
|
|
|
// A @ secondBatchTimestamp right window created when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchRightWindow, Long.MAX_VALUE)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), 1, thirdBatchTimestamp),
|
|
|
|
|
// A @ secondBatchTimestamp left window created when A @ secondBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchLeftWindow, Long.MAX_VALUE)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), 2, secondBatchTimestamp),
|
|
|
|
|
// A @ firstBatchTimestamp right window updated when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 2, thirdBatchTimestamp),
|
|
|
|
|
// A @ thirdBatchTimestamp left window created when A @ thirdBatchTimestamp processed
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(thirdBatchLeftWindow, Long.MAX_VALUE)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("A", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), 3, thirdBatchTimestamp),
|
|
|
|
|
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchLeftWindow, Long.MAX_VALUE)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchRightWindow, Long.MAX_VALUE)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchLeftWindow, Long.MAX_VALUE)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(thirdBatchLeftWindow, Long.MAX_VALUE)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchLeftWindow, Long.MAX_VALUE)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchRightWindow, Long.MAX_VALUE)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchLeftWindow, Long.MAX_VALUE)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(thirdBatchLeftWindow, Long.MAX_VALUE)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchLeftWindow, Long.MAX_VALUE)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchRightWindow, Long.MAX_VALUE)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchLeftWindow, Long.MAX_VALUE)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(thirdBatchLeftWindow, Long.MAX_VALUE)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchLeftWindow, Long.MAX_VALUE)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchRightWindow, Long.MAX_VALUE)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchLeftWindow, Long.MAX_VALUE)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindow, Long.MAX_VALUE)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(thirdBatchLeftWindow, Long.MAX_VALUE)), 3, thirdBatchTimestamp)
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("B", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("C", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("D", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), 3, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchLeftWindowStart, firstBatchLeftWindowEnd)), 1, firstBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 1, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchRightWindowStart, secondBatchRightWindowEnd)), 1, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(secondBatchLeftWindowStart, secondBatchLeftWindowEnd)), 2, secondBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(firstBatchRightWindowStart, firstBatchRightWindowEnd)), 2, thirdBatchTimestamp),
|
|
|
|
|
new KeyValueTimestamp<>(new Windowed<>("E", new TimeWindow(thirdBatchLeftWindowStart, thirdBatchLeftWindowEnd)), 3, thirdBatchTimestamp)
|
|
|
|
|
);
|
|
|
|
|
|
|
|
|
|
assertThat(windowedMessages, is(expectResult));
|
|
|
|
@ -1030,6 +1045,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
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());
|
|
|
|
|
consumerProperties.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, 500L);
|
|
|
|
|
if (keyDeserializer instanceof TimeWindowedDeserializer || keyDeserializer instanceof SessionWindowedDeserializer) {
|
|
|
|
|
consumerProperties.setProperty(StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS,
|
|
|
|
|
Serdes.serdeFrom(innerClass).getClass().getName());
|
|
|
|
@ -1052,6 +1068,7 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
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());
|
|
|
|
|
consumerProperties.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, 500L);
|
|
|
|
|
if (keyDeserializer instanceof TimeWindowedDeserializer || keyDeserializer instanceof SessionWindowedDeserializer) {
|
|
|
|
|
consumerProperties.setProperty(StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS,
|
|
|
|
|
Serdes.serdeFrom(innerClass).getClass().getName());
|
|
|
|
@ -1085,7 +1102,8 @@ public class KStreamAggregationIntegrationTest {
|
|
|
|
|
"--property", "key.deserializer=" + keyDeserializer.getClass().getName(),
|
|
|
|
|
"--property", "value.deserializer=" + valueDeserializer.getClass().getName(),
|
|
|
|
|
"--property", "key.separator=" + keySeparator,
|
|
|
|
|
"--property", "key.deserializer." + StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName()
|
|
|
|
|
"--property", "key.deserializer." + StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(),
|
|
|
|
|
"--property", "key.deserializer.window.size.ms=500",
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
ConsoleConsumer.messageCount_$eq(0); //reset the message count
|
|
|
|
|