junrao commented on a change in pull request #10405: URL: https://github.com/apache/kafka/pull/10405#discussion_r606022097
########## File path: core/src/main/scala/kafka/log/LogSegment.scala ########## @@ -59,7 +60,8 @@ class LogSegment private[log] (val log: FileRecords, val baseOffset: Long, val indexIntervalBytes: Int, val rollJitterMs: Long, - val time: Time) extends Logging { + val time: Time, + val needsFlushParentDir: Boolean = false) extends Logging { Review comment: Could we add the new param to the javadoc? ########## File path: core/src/main/scala/kafka/log/LogSegment.scala ########## @@ -95,6 +97,9 @@ class LogSegment private[log] (val log: FileRecords, /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 + /* whether or not we need to flush the parent dir during flush */ Review comment: during flush => during the first flush ? ########## File path: clients/src/main/java/org/apache/kafka/common/record/FileRecords.java ########## @@ -433,7 +440,8 @@ public static FileRecords open(File file, public static FileRecords open(File file, boolean fileAlreadyExists, int initFileSize, - boolean preallocate) throws IOException { + boolean preallocate, + boolean needsRecovery) throws IOException { Review comment: This change seems unneeded? ########## File path: core/src/main/scala/kafka/log/LogSegment.scala ########## @@ -657,17 +668,19 @@ class LogSegment private[log] (val log: FileRecords, object LogSegment { def open(dir: File, baseOffset: Long, config: LogConfig, time: Time, fileAlreadyExists: Boolean = false, - initFileSize: Int = 0, preallocate: Boolean = false, fileSuffix: String = ""): LogSegment = { + initFileSize: Int = 0, preallocate: Boolean = false, fileSuffix: String = "", + needsRecovery: Boolean = true): LogSegment = { Review comment: It seems that needsRecovery should default to false? -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org