cadonna commented on code in PR #15361:
URL: https://github.com/apache/kafka/pull/15361#discussion_r1505538908
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java:
##########
@@ -259,19 +251,14 @@ void initialize() {
for (final Map.Entry<TopicPartition, Long> entry :
partitionOffsets.entrySet()) {
globalConsumer.seek(entry.getKey(), entry.getValue());
}
- lastFlush = time.milliseconds();
}
void pollAndUpdate() {
final ConsumerRecords<byte[], byte[]> received =
globalConsumer.poll(pollTime);
for (final ConsumerRecord<byte[], byte[]> record : received) {
stateMaintainer.update(record);
}
- final long now = time.milliseconds();
- if (now - flushInterval >= lastFlush) {
- stateMaintainer.flushState();
- lastFlush = now;
- }
+ stateMaintainer.maybeCheckpoint();
Review Comment:
No, checkpointing the local store is not only based on the delta. Method
`maybeCheckpoint()` is called after a commit and a commit is done in regular
time-intervals.
https://github.com/apache/kafka/blob/d144b7ee387308a59e52cbdabc7b66dd3b2926cc/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java#L513
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java:
##########
@@ -217,21 +215,53 @@ public void shouldFlushStateManagerWithOffsets() {
final Map<TopicPartition, Long> expectedOffsets = new HashMap<>();
expectedOffsets.put(t1, 52L);
expectedOffsets.put(t2, 100L);
+
globalStateTask.initialize();
globalStateTask.update(record(topic1, 1, 51, "foo".getBytes(),
"foo".getBytes()));
globalStateTask.flushState();
+
assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+ assertTrue(stateMgr.flushed);
}
@Test
public void shouldCheckpointOffsetsWhenStateIsFlushed() {
final Map<TopicPartition, Long> expectedOffsets = new HashMap<>();
expectedOffsets.put(t1, 102L);
expectedOffsets.put(t2, 100L);
+
globalStateTask.initialize();
globalStateTask.update(record(topic1, 1, 101, "foo".getBytes(),
"foo".getBytes()));
globalStateTask.flushState();
- assertThat(stateMgr.changelogOffsets(), equalTo(expectedOffsets));
+
+ assertEquals(expectedOffsets, stateMgr.changelogOffsets());
+ assertTrue(stateMgr.checkpointWritten);
+ }
+
+ @Test
+ public void shouldNotCheckpointIfNotReceivedEnoughRecords() {
+ globalStateTask.initialize();
+ globalStateTask.update(record(topic1, 1, 9000L, "foo".getBytes(),
"foo".getBytes()));
+ globalStateTask.maybeCheckpoint();
+
+ assertEquals(offsets, stateMgr.changelogOffsets());
+ assertFalse(stateMgr.flushed);
+ assertFalse(stateMgr.checkpointWritten);
+ }
+
+ @Test
+ public void shouldCheckpointIfReceivedEnoughRecords() {
+ final Map<TopicPartition, Long> expectedOffsets = new HashMap<>();
+ expectedOffsets.put(t1, 10051L); // t1 advanced with 10001 records
+ expectedOffsets.put(t2, 100L);
+
+ globalStateTask.initialize();
+ globalStateTask.update(record(topic1, 1, 10050L, "foo".getBytes(),
"foo".getBytes()));
Review Comment:
I know, I wanted to have a verification for the boundary. Could you please
re-add the update with offset 9000 and additionally do an update with offset
10049. Then we also test the case with multiple updates before the checkpoint.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]