mjsax commented on a change in pull request #10170:
URL: https://github.com/apache/kafka/pull/10170#discussion_r581324176
##########
File path:
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
##########
@@ -1805,18 +1806,16 @@ public void shouldPunctuateActiveTask() {
final List<Long> punctuatedStreamTime = new ArrayList<>();
final List<Long> punctuatedWallClockTime = new ArrayList<>();
final org.apache.kafka.streams.processor.ProcessorSupplier<Object,
Object> punctuateProcessor =
- () -> new org.apache.kafka.streams.processor.Processor<Object,
Object>() {
+ () -> new
org.apache.kafka.streams.processor.AbstractProcessor<Object, Object>() {
@Override
public void init(final
org.apache.kafka.streams.processor.ProcessorContext context) {
context.schedule(Duration.ofMillis(100L),
PunctuationType.STREAM_TIME, punctuatedStreamTime::add);
context.schedule(Duration.ofMillis(100L),
PunctuationType.WALL_CLOCK_TIME, punctuatedWallClockTime::add);
+ context.schedule(Duration.ofMillis(200L),
PunctuationType.WALL_CLOCK_TIME, timestamp -> context.forward("key", "value"));
Review comment:
What is the purpose of adding this line? Seem this case is covered in
the newly added test below?
##########
File path:
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
##########
@@ -1874,6 +1873,85 @@ public void close() {}
assertEquals(2, punctuatedWallClockTime.size());
}
+ @Test
+ public void shouldPunctuateWithTimestampPreservedInProcessorContext() {
+ final org.apache.kafka.streams.kstream.TransformerSupplier<Object,
Object, KeyValue<Object, Object>> punctuateProcessor =
+ () -> new org.apache.kafka.streams.kstream.Transformer<Object,
Object, KeyValue<Object, Object>>() {
+ @Override
+ public void init(final
org.apache.kafka.streams.processor.ProcessorContext context) {
+ context.schedule(Duration.ofMillis(100L),
PunctuationType.WALL_CLOCK_TIME, timestamp -> context.forward("key", "value"));
+ context.schedule(Duration.ofMillis(100L),
PunctuationType.STREAM_TIME, timestamp -> context.forward("key", "value"));
+ }
+
+ @Override
+ public KeyValue<Object, Object> transform(final Object key,
final Object value) {
+ return null;
+ }
+
+ @Override
+ public void close() {}
+ };
+
+ final List<Long> peekedContextTime = new ArrayList<>();
+ final org.apache.kafka.streams.processor.ProcessorSupplier<Object,
Object> peekProcessor =
+ () -> new
org.apache.kafka.streams.processor.AbstractProcessor<Object, Object>() {
+ @Override
+ public void process(final Object key, final Object value) {
+ peekedContextTime.add(context.timestamp());
+ }
+ };
+
+ internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
+ .transform(punctuateProcessor)
+ .process(peekProcessor);
+ internalStreamsBuilder.buildAndOptimizeTopology();
+
+ final long currTime = mockTime.milliseconds();
+ final StreamThread thread = createStreamThread(CLIENT_ID, config,
false);
+
+ thread.setState(StreamThread.State.STARTING);
+ thread.rebalanceListener().onPartitionsRevoked(Collections.emptySet());
+ final List<TopicPartition> assignedPartitions = new ArrayList<>();
+
+ final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
+
+ // assign single partition
+ assignedPartitions.add(t1p1);
+ activeTasks.put(task1, Collections.singleton(t1p1));
+
+ thread.taskManager().handleAssignment(activeTasks, emptyMap());
+
+ clientSupplier.consumer.assign(assignedPartitions);
+
clientSupplier.consumer.updateBeginningOffsets(Collections.singletonMap(t1p1,
0L));
+ thread.rebalanceListener().onPartitionsAssigned(assignedPartitions);
+
+ thread.runOnce();
+ assertEquals(0, peekedContextTime.size());
+
+ mockTime.sleep(100L);
+ thread.runOnce();
+
+ assertEquals(1, peekedContextTime.size());
+ assertEquals(currTime + 100L, peekedContextTime.get(0).longValue());
+
+ clientSupplier.consumer.addRecord(new ConsumerRecord<>(
+ topic1,
+ 1,
+ 0L,
+ 100L,
+ TimestampType.CREATE_TIME,
+ ConsumerRecord.NULL_CHECKSUM,
+ "K".getBytes().length,
+ "V".getBytes().length,
+ "K".getBytes(),
+ "V".getBytes()));
+
+ thread.runOnce();
+
+ assertEquals(2, peekedContextTime.size());
+ assertEquals(0L, peekedContextTime.get(1).longValue());
Review comment:
Why is the expected value `0L` ? Should it not be `100L` (ie, the input
record's timestamp that dictates stream-time)?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]