Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-12575: Eliminate Log.isLogDirOffline boolean attribute #10430

Merged
merged 2 commits into from
Mar 31, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 4 additions & 13 deletions core/src/main/scala/kafka/log/Log.scala
Original file line number Diff line number Diff line change
Expand Up @@ -283,10 +283,6 @@ class Log(@volatile private var _dir: File,

@volatile private var nextOffsetMetadata: LogOffsetMetadata = _

// Log dir failure is handled asynchronously we need to prevent threads
// from reading inconsistent state caused by a failure in another thread
@volatile private var logDirOffline = false

/* The earliest offset which is part of an incomplete transaction. This is used to compute the
* last stable offset (LSO) in ReplicaManager. Note that it is possible that the "true" first unstable offset
* gets removed from the log (through record or segment deletion). In this case, the first unstable offset
Expand Down Expand Up @@ -1332,12 +1328,6 @@ class Log(@volatile private var _dir: File,
}
}

private def checkForLogDirFailure(): Unit = {
if (logDirOffline) {
throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
}
}

def maybeAssignEpochStartOffset(leaderEpoch: Int, startOffset: Long): Unit = {
leaderEpochCache.foreach { cache =>
cache.assign(leaderEpoch, startOffset)
Expand Down Expand Up @@ -2454,13 +2444,14 @@ class Log(@volatile private var _dir: File,
private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)

private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
if (logDirFailureChannel.hasOfflineLogDir(parentDir)) {
throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
}
try {
checkForLogDirFailure()
fun
} catch {
case e: IOException =>
logDirOffline = true
logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
logDirFailureChannel.maybeAddOfflineLogDir(parentDir, msg, e)
throw new KafkaStorageException(msg, e)
}
}
Expand Down
4 changes: 4 additions & 0 deletions core/src/main/scala/kafka/server/LogDirFailureChannel.scala
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ class LogDirFailureChannel(logDirNum: Int) extends Logging {
private val offlineLogDirs = new ConcurrentHashMap[String, String]
private val offlineLogDirQueue = new ArrayBlockingQueue[String](logDirNum)

def hasOfflineLogDir(logDir: String): Boolean = {
offlineLogDirs.containsKey(logDir)
}

/*
* If the given logDir is not already offline, add it to the
* set of offline log dirs and enqueue it to the logDirFailureEvent queue
Expand Down