junrao commented on a change in pull request #11345: URL: https://github.com/apache/kafka/pull/11345#discussion_r781485562
########## File path: core/src/main/scala/kafka/log/UnifiedLog.scala ########## @@ -1498,28 +1498,44 @@ 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-1. Review comment: The comment is inaccurate. The recovery point is always offset. ########## File path: core/src/main/scala/kafka/log/LogLoader.scala ########## @@ -333,8 +333,9 @@ object LogLoader extends Logging { try segment.sanityCheck(timeIndexFileNewlyCreated) catch { case _: NoSuchFileException => - error(s"${params.logIdentifier}Could not find offset index file corresponding to log file" + - s" ${segment.log.file.getAbsolutePath}, recovering segment and rebuilding index files...") + if (segment.baseOffset < params.recoveryPointCheckpoint) Review comment: This condition is correct if hadCleanShutdown is false. If hadCleanShutdown is true, it seems the condition should be `segment.baseOffset <=params.recoveryPointCheckpoint`. Or maybe we should just always log the error if hadCleanShutdown is true. ########## File path: core/src/test/scala/unit/kafka/log/LogLoaderTest.scala ########## @@ -1677,4 +1677,46 @@ class LogLoaderTest { s"Found offsets with missing producer state snapshot files: $offsetsWithMissingSnapshotFiles") assertFalse(logDir.list().exists(_.endsWith(UnifiedLog.DeletedFileSuffix)), "Expected no files to be present with the deleted file suffix") } + + @Test + def testRecoverWithEmptyActiveSegment(): Unit = { + val numMessages = 100 + val messageSize = 100 + val segmentSize = 7 * messageSize + val indexInterval = 3 * messageSize + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), + timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + assertEquals(numMessages, log.logEndOffset, + "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) + log.roll() + log.flush(false) + assertThrows(classOf[NoSuchFileException], () => log.activeSegment.sanityCheck(true)) + var lastOffset = log.logEndOffset + + log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) + assertEquals(lastOffset, log.recoveryPoint, s"Unexpected recovery point") + assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery") + assertEquals(0, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.") + log.activeSegment.sanityCheck(true) // this should not throw Review comment: Could we add a comment why this check won't throw after re-instantiating the log? ########## File path: core/src/main/scala/kafka/log/UnifiedLog.scala ########## @@ -1498,28 +1498,44 @@ 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-1. + * + * @param offset The offset to flush up to (non-inclusive); the new recovery point Review comment: Should we get rid of "(non-inclusive)"? ########## File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java ########## @@ -184,8 +184,10 @@ default ValidOffsetAndEpoch validateOffsetAndEpoch(long offset, int epoch) { /** * Flush the current log to disk. + * + * @param inclusive Whether the flush includes the log end offset. Should be `true` during close; otherwise false. */ - void flush(); + void flush(boolean inclusive); Review comment: We use forceFlushActiveSegment in UnifiedLog.flush(). Should we be consistent with the name? ########## File path: core/src/test/scala/unit/kafka/log/LogLoaderTest.scala ########## @@ -1677,4 +1677,46 @@ class LogLoaderTest { s"Found offsets with missing producer state snapshot files: $offsetsWithMissingSnapshotFiles") assertFalse(logDir.list().exists(_.endsWith(UnifiedLog.DeletedFileSuffix)), "Expected no files to be present with the deleted file suffix") } + + @Test + def testRecoverWithEmptyActiveSegment(): Unit = { + val numMessages = 100 + val messageSize = 100 + val segmentSize = 7 * messageSize + val indexInterval = 3 * messageSize + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), + timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + assertEquals(numMessages, log.logEndOffset, + "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) + log.roll() + log.flush(false) + assertThrows(classOf[NoSuchFileException], () => log.activeSegment.sanityCheck(true)) + var lastOffset = log.logEndOffset + + log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) Review comment: Should we call log.closeHandlers() before assigning a new value to log? Otherwise, it seems that we are leaking file handles. ########## File path: core/src/main/scala/kafka/log/UnifiedLog.scala ########## @@ -1498,28 +1498,44 @@ 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-1. + * + * @param offset The offset to flush up to (non-inclusive); 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 $flushOffset and recovery point $newRecoveryPoint") { Review comment: Instead of $flushOffset, perhaps it's clearer to use "$offset(ex/inclusive)"? Ditto for the debug logging below. ########## File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala ########## @@ -211,8 +211,8 @@ final class KafkaMetadataLog private ( new LogOffsetMetadata(hwm.messageOffset, segmentPosition) } - override def flush(): Unit = { - log.flush() + override def flush(inclusive: Boolean): Unit = { Review comment: Should inclusive be renamed to forceFlushActiveSegment? -- 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