KAFKA-13603: Allow the empty active segment to have missing offset index during recovery#11345
KAFKA-13603: Allow the empty active segment to have missing offset index during recovery#11345junrao merged 33 commits intoapache:trunkfrom
Conversation
|
cc @kowshik |
|
@ccding the PR does have unit tests though, so we should update that part of the PR message. |
|
Updated the Test section |
Within a LogSegment, the TimeIndex and OffsetIndex are lazy indices that don't get created on disk until they are accessed for the first time. However, Log recovery logic expects the presence of offset index file on disk for each segment, otherwise the segment is considered corrupted. Author: Kowshik Prakasam <kowshik@gmail.com>
|
@kowshik Thanks for the code review. Updated your email in the PR description as well as in the commit message. |
|
Failed tests are irrelevant and passed on my local run. |
| if (lazyOffsetIndex.file.exists) { | ||
| def sanityCheck(timeIndexFileNewlyCreated: Boolean, isActiveSegment: Boolean): Unit = { | ||
| // We allow for absence of offset index file only for an empty active segment. | ||
| if ((isActiveSegment && size == 0) || lazyOffsetIndex.file.exists) { |
There was a problem hiding this comment.
I am wondering why the active segment will be missing the offset index file during a clean shutdown. When we load the segments during broker restart, we call resizeIndexes() on the last segment. This should trigger the creation of the offset index file, which will be flushed on broker shutdown.
There was a problem hiding this comment.
When we load the segments during broker restart, we call resizeIndexes() on the last segment. This should trigger the creation of the offset index file, which will be flushed on broker shutdown.
The sanityCheck is called before resizeIndexes.
It appears you are talking about we start the broker then immediately shut it down. In between the active segment may have been changed, and if the new one is empty, no index file is created.
There was a problem hiding this comment.
I am still trying to understand if the missing index is the result of a clean shutdown or a hard shutdown. When will roll a segment, the index on the new active segment is created lazily. However, during a clean shutdown, we force flush the active segment, which should trigger the creation of an empty index file because the following method is used in segment flush.
def offsetIndex: OffsetIndex = lazyOffsetIndex.get
On a hard shutdown, it's possible for the offset index to be missing. However, in that case, the offset index can be missing even when the log is not empty. So, I am wondering how common of an issue that we are fixing.
There was a problem hiding this comment.
@junrao: When the UnifiedLog is flushed during clean shutdown, we flush the LocaLog until the logEndOffset. Here an empty active segment is not included in the list of candidate segments to be flushed. The reason is that during LocalLog.flush(), the LogSegments.values(recoveryPoint, logEndOffset) call here does not select the empty active segment (doc), because, the logEndOffset would match the base offset of the empty active segment and thus get ommitted. So, prior to clean shutdown if the empty active segment's offset index was never created before, then, the offset index will not be created during clean shutdown because the empty active segment is never flushed.
The above is shown in the following passing unit test:
@Test
def testFlushEmptyActiveSegmentDoesNotCreateOffsetIndex(): Unit = {
// Create an empty log.
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024)
val log = createLog(logDir, logConfig)
val oneRecord = TestUtils.records(List(
new SimpleRecord(mockTime.milliseconds, "a".getBytes, "value".getBytes)
))
// Append a record and flush. Verify that there exists only 1 segment.
log.appendAsLeader(oneRecord, leaderEpoch = 0)
assertEquals(1, log.logEndOffset)
log.flush()
assertEquals(1, log.logSegments.size)
assertTrue(UnifiedLog.logFile(logDir, 0).exists())
assertTrue(UnifiedLog.offsetIndexFile(logDir, 0).exists())
assertFalse(UnifiedLog.logFile(logDir, 1).exists())
assertFalse(UnifiedLog.offsetIndexFile(logDir, 1).exists())
// Roll the log and verify that the new active segment's offset index is missing.
log.roll()
assertEquals(2, log.logSegments.size)
assertTrue(UnifiedLog.logFile(logDir, 0).exists())
assertTrue(UnifiedLog.offsetIndexFile(logDir, 0).exists())
assertTrue(UnifiedLog.logFile(logDir, 1).exists())
assertFalse(UnifiedLog.offsetIndexFile(logDir, 1).exists())
// Flush the log and once again verify that the active segment's offset index is still missing.
log.flush()
assertTrue(UnifiedLog.logFile(logDir, 0).exists())
assertTrue(UnifiedLog.offsetIndexFile(logDir, 0).exists())
assertTrue(UnifiedLog.logFile(logDir, 1).exists())
assertFalse(UnifiedLog.offsetIndexFile(logDir, 1).exists())
// Close the log and verify that the active segment's offset index is still missing.
log.close()
assertTrue(UnifiedLog.logFile(logDir, 0).exists())
assertTrue(UnifiedLog.offsetIndexFile(logDir, 0).exists())
assertTrue(UnifiedLog.logFile(logDir, 1).exists())
assertFalse(UnifiedLog.offsetIndexFile(logDir, 1).exists())
}
This PR mainly fixes a logging issue in the code. For example, one situation where the issue happens more frequently is the following: Imagine there exists a topic with very low ingress traffic in some/all partitions. Imagine that for this topic the retention setting causes all existing segments to expire and get removed. In such a case, we roll the log to create an active segment. This ensures there is at least one segment remaining in the LocalLog when the retention loop completes. However we don't create the offset index for the active segment until the first append operation. Now before the first append, if the Kafka cluster is rolled then we will see this false negative corruption error message during recovery.
This PR fixes the logging problem by ignoring the absence of offset index for an empty active segment during recovery.
There was a problem hiding this comment.
@kowshik : Thanks for the great explanation. It makes sense to me now.
If we don't flush the only empty log segment during clean shutdown, we could lose the log segment file as well, which causes the replica to lose track of logEndOffset. I am wondering if we should force flush the empty log segment during clean shutdown too.
There was a problem hiding this comment.
Will def flush(): Unit = flush(logEndOffset + 1) trigger flushing empty active segments every time we roll a segment, not only during shutdown?
There was a problem hiding this comment.
@kowshik : #8346 tries to avoid opening the index during close() when the index is not opened yet. This applies to existing segments on broker restart. For active segment, we typically need to open the index anyway. So, we probably don't need to optimize the rare case when it's empty. Plus, the danger of losing logEndOffset is a bigger concern than avoiding the cost of opening one index file.
There was a problem hiding this comment.
Updated. Please let me know if I misunderstood anything.
| * Flush all local log segments | ||
| */ | ||
| def flush(): Unit = flush(logEndOffset) | ||
| def flush(): Unit = flush(logEndOffset + 1) |
There was a problem hiding this comment.
Could we add a comment why we need to flush to logEndOffset + 1? Also, could we have a test that verifies the index file is present after an empty segment is rolled and the broker is shut down?
There was a problem hiding this comment.
Thanks for the comment. Will write it later. Was too busy yesterday and didn't have time to do so.
| * Flush all local log segments | ||
| */ | ||
| def flush(): Unit = flush(logEndOffset) | ||
| def flush(): Unit = flush(logEndOffset + 1) |
There was a problem hiding this comment.
Can we add unit test coverage for this change in UnifiedLogTest.scala?
|
While I am working on the test, I have a question: The They are not during a shutdown. Will these also flush empty active segments and hurt performance? |
|
@ccding During a call to Furthermore, we typically don't configure to flush the log periodically or during appends. Even then, when we call (1) Periodic flush: kafka/core/src/main/scala/kafka/log/LogManager.scala Lines 1244 to 1246 in 0fe4e24 Here, (2) Flush during appends: Here, |
|
@ccding Are you planning to add a unit test for this PR, and address the recent review comments? |
|
@kowshik I will find some time to work on it |
|
The change failed recovery point check at I am not sure what to do. Please advise. |
|
@ccding : Thanks for reporting the test failure. This brings up a good point. It's kind of weird to ever have recovery point > log end offset. I am thinking that another potential way to fix this is for the flush() call in close() from recovery point to log end offset to be inclusive on both ends. This way, a flush of (log end offset, log end offset) will force a flush since it's needed for flushing the metadata of log end offset. All other flushes will still be exclusive on the right end since they don't need to preserve the metadata on the right end. |
|
|
||
| /** | ||
| * Flush all local log segments | ||
| * We have to pass logEngOffset + 1 to the `def flush(offset: Long): Unit` function to flush empty |
There was a problem hiding this comment.
This comment is in the wrong place.
| def close(): Unit = { | ||
| debug("Closing log") | ||
| lock synchronized { | ||
| flush(logEndOffset + 1) |
There was a problem hiding this comment.
This seems to have the same problem that the log recovery point could be moved to logEndOffset + 1, which is a bit weird?
There was a problem hiding this comment.
Actually, this doesn't work. I am trying to figure out a proper solution.
|
@kowshik @junrao @hachikuji I have addressed the review comments and please take a look. thanks |
|
ping @kowshik @junrao @hachikuji |
|
|
||
| override def flush(): Unit = { | ||
| log.flush() | ||
| override def flush(inclusive: Boolean): Unit = { |
There was a problem hiding this comment.
Should inclusive be renamed to forceFlushActiveSegment?
|
|
||
| /** | ||
| * 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. |
There was a problem hiding this comment.
The comment is inaccurate. The recovery point is always offset.
| * 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 |
There was a problem hiding this comment.
Should we get rid of "(non-inclusive)"?
| 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") { |
There was a problem hiding this comment.
Instead of $flushOffset, perhaps it's clearer to use "$offset(ex/inclusive)"? Ditto for the debug logging below.
| 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) |
There was a problem hiding this 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.
There was a problem hiding this comment.
I think if hadCleanShutdown is true, it should never throw NoSuchFileException unless there is a bug in the code.
Added the hadCleanShutdown check anyways to catch potential issues: if (params.hadCleanShutdown || segment.baseOffset < params.recoveryPointCheckpoint)
| * @param inclusive Whether the flush includes the log end offset. Should be `true` during close; otherwise false. | ||
| */ | ||
| void flush(); | ||
| void flush(boolean inclusive); |
There was a problem hiding this comment.
We use forceFlushActiveSegment in UnifiedLog.flush(). Should we be consistent with the name?
| 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 |
There was a problem hiding this comment.
Could we add a comment why this check won't throw after re-instantiating the log?
| assertThrows(classOf[NoSuchFileException], () => log.activeSegment.sanityCheck(true)) | ||
| var lastOffset = log.logEndOffset | ||
|
|
||
| log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) |
There was a problem hiding this comment.
Should we call log.closeHandlers() before assigning a new value to log? Otherwise, it seems that we are leaking file handles.
| + { if (includingOffset) "inclusive" else "exclusive" } | ||
| + s") and recovery point $newRecoveryPoint") { | ||
| if (flushOffset > localLog.recoveryPoint) { | ||
| debug(s"Flushing log up to offset $flushOffset with recovery point $newRecoveryPoint, last flushed: $lastFlushTime, current time: ${time.milliseconds()}, " + |
There was a problem hiding this comment.
Instead of $flushOffset, could we change to "$offset(ex/inclusive)"?
| /** | ||
| * Flush the current log to disk. | ||
| * | ||
| * @param forceFlushActiveSegment Whether the flush includes the log end offset. Should be `true` during close; otherwise false. |
There was a problem hiding this comment.
Whether the flush includes the log end offset => Whether to force flush the active segment?
junrao
left a comment
There was a problem hiding this comment.
@ccding : Thanks for the updated PR. LGTM
@hachikuji : Any other comments from you?
|
@ccding : The PR is kind of large now. Could you associate the PR with a jira? Thanks. |
| s") and recovery point $newRecoveryPoint") { | ||
| if (flushOffset > localLog.recoveryPoint) { | ||
| debug(s"Flushing log up to offset (" + | ||
| { if (includingOffset) "inclusive" else "exclusive" } + |
There was a problem hiding this 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.
|
@ccding : There is a conflict now. Could you rebase? |
| 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, " + | ||
| s"includingOffset=$includingOffset, newRecoveryPoint=$newRecoveryPoint") { |
There was a problem hiding this comment.
I think the logging that you had before the last commit is more intuitive.
kowshik
left a comment
There was a problem hiding this comment.
Thanks for the updated PR. Have a small comment below.
| if (flushOffset > localLog.recoveryPoint) { | ||
| debug(s"Flushing log up to offset=$offset, includingOffset=$includingOffset, " + | ||
| s"newRecoveryPoint=$newRecoveryPoint, last flushed: $lastFlushTime, current time: ${time.milliseconds()}, " + | ||
| debug(s"Flushing log up to offset ($includingOffsetStr)" + |
There was a problem hiding this comment.
I think perhaps you meant to use offset=$offset?
|
@ccding : Thanks for the latest PR. LGTM. Merged to trunk. |
Within a LogSegment, the TimeIndex and OffsetIndex are lazy indices that don't get created on disk until they are accessed for the first time. However, Log recovery logic expects the presence of an offset index file on disk for each segment, otherwise, the segment is considered corrupted.
This PR introduces a
forceFlushActiveSegmentboolean for thelog.flushfunction to allow the shutdown process to flush the empty active segment, which makes sure the offset index file exists.Co-Author: Kowshik Prakasam kowshik@gmail.com
Committer Checklist (excluded from commit message)