jeffkbkim commented on code in PR #15361: URL: https://github.com/apache/kafka/pull/15361#discussion_r1490286153
########## streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java: ########## @@ -161,6 +140,10 @@ public void update(final ConsumerRecord<byte[], byte[]> record) { updatedPartitions.put(tp, updatedPartitions.get(tp) + 1); } + @Override + public void maybeCheckpoint() { + flushState(); + } } } Review Comment: nit: can we add a newline? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java: ########## @@ -150,5 +150,11 @@ private void initTopology() { } } + @Override + public void maybeCheckpoint() { + if (StateManagerUtil.checkpointNeeded(false, stateMgr.changelogOffsets(), offsets)) { Review Comment: [AbstractTask#maybeCheckpoint()](https://github.com/apache/kafka/blob/a1f3c6d16061566a4f53c72a95e2679b8ee229e0/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java#L97) passes in `stateMgr.changelogOffsets` as the new offset snapshot in `StateManagerUtil#checkpointNeeded`. Why is the behavior reversed here? is it because `offsets` is the object that is updated on every `update()`? what about `ProcessorStateManager#changelogOffsets`, would that not have the latest? ########## 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: should we still keep an interval-based flush or is the delta check sufficient? -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org