You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/08/03 16:34:06 UTC

[GitHub] [kafka] junrao commented on a change in pull request #8936: KAFKA-10207: Fixed padded timeindex causing premature data deletion

junrao commented on a change in pull request #8936:
URL: https://github.com/apache/kafka/pull/8936#discussion_r463885952



##########
File path: core/src/main/scala/kafka/log/LogSegment.scala
##########
@@ -60,11 +60,34 @@ 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 segmentRecovery: LogSegment => Int) extends Logging {
 
-  def offsetIndex: OffsetIndex = lazyOffsetIndex.get
+  def loadIndexWithRecovery[T <: AbstractIndex](lazyIndex: LazyIndex[T]): T = {

Review comment:
       Could this be private?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -652,6 +653,12 @@ class Log(@volatile private var _dir: File,
     bytesTruncated
   }
 
+  def segmentRecovery(): LogSegment => Int = {
+    (segment: LogSegment) => {
+      recoverSegment(segment, None)

Review comment:
       Log.recoverSegment() updates the producerState. If we are just rebuilding the indexes, we don't need to change producerState. Perhaps we could introduce a new method LogSegment.rebuildIndex() that does the same index rebuilding logic as LogSegment.recovery(), but without touching the producerState and leaderEpoch. If we encounter CorruptRecordException or InvalidRecordException, we reset the recovery point and fail the broker.

##########
File path: core/src/main/scala/kafka/log/LazyIndex.scala
##########
@@ -52,6 +52,13 @@ class LazyIndex[T <: AbstractIndex] private (@volatile private var indexWrapper:
 
   def file: File = indexWrapper.file
 
+  def isLoaded: Boolean = {
+    indexWrapper match {
+      case indexValue: IndexValue[T] => true

Review comment:
       Since indexValue is not used, we could just do `case _: IndexValue[T]`.




----------------------------------------------------------------
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:
users@infra.apache.org