kowshik commented on a change in pull request #11345: URL: https://github.com/apache/kafka/pull/11345#discussion_r791176937
########## File path: core/src/main/scala/kafka/log/UnifiedLog.scala ########## @@ -1498,28 +1498,48 @@ class UnifiedLog(@volatile var logStartOffset: Long, producerStateManager.takeSnapshot() updateHighWatermarkWithLogEndOffset() // Schedule an asynchronous flush of the old segment - scheduler.schedule("flush-log", () => flush(newSegment.baseOffset)) + scheduler.schedule("flush-log", () => flushUptoOffsetExclusive(newSegment.baseOffset)) newSegment } /** * Flush all local log segments + * + * @param forceFlushActiveSegment should be true during a clean shutdown, and false otherwise. The reason is that + * we have to pass logEndOffset + 1 to the `localLog.flush(offset: Long): Unit` function to flush empty + * active segments, which is important to make sure we persist the active segment file during shutdown, particularly + * when it's empty. */ - def flush(): Unit = flush(logEndOffset) + def flush(forceFlushActiveSegment: Boolean): Unit = flush(logEndOffset, forceFlushActiveSegment) /** * Flush local log segments for all offsets up to offset-1 * * @param offset The offset to flush up to (non-inclusive); the new recovery point */ - def flush(offset: Long): Unit = { - maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") { - if (offset > localLog.recoveryPoint) { - debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime, current time: ${time.milliseconds()}, " + + def flushUptoOffsetExclusive(offset: Long): Unit = flush(offset, false) + + /** + * Flush local log segments for all offsets up to offset-1 if includingOffset=false; up to offset + * if includingOffset=true. The recovery point is set to offset. + * + * @param offset The offset to flush up to; the new recovery point + * @param includingOffset Whether the flush includes the provided offset. + */ + private def flush(offset: Long, includingOffset: Boolean): Unit = { + val flushOffset = if (includingOffset) offset + 1 else offset + val newRecoveryPoint = offset + maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset (" + + { if (includingOffset) "inclusive" else "exclusive" } + + s") and recovery point $newRecoveryPoint") { + if (flushOffset > localLog.recoveryPoint) { + debug(s"Flushing log up to offset (" + + { if (includingOffset) "inclusive" else "exclusive" } + Review comment: The clause `if (includingOffset) "inclusive" else "exclusive" }` is redundant. This can be extracted into a separate variable, or just eliminate it and instead you could just print the value of `includingOffset`. -- 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