hachikuji commented on a change in pull request #9816:
URL: https://github.com/apache/kafka/pull/9816#discussion_r566334408



##########
File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
##########
@@ -66,32 +78,67 @@ class KafkaMetadataLog(
     if (records.sizeInBytes == 0)
       throw new IllegalArgumentException("Attempt to append an empty record 
set")
 
-    val appendInfo = log.appendAsLeader(records.asInstanceOf[MemoryRecords],
-      leaderEpoch = epoch,
-      origin = AppendOrigin.Coordinator)
-    new LogAppendInfo(appendInfo.firstOffset.getOrElse {
-      throw new KafkaException("Append failed unexpectedly")
-    }, appendInfo.lastOffset)
+    handleAndConvertLogAppendInfo(
+      log.appendAsLeader(records.asInstanceOf[MemoryRecords],
+        leaderEpoch = epoch,
+        origin = AppendOrigin.Coordinator
+      )
+    )
   }
 
   override def appendAsFollower(records: Records): LogAppendInfo = {
     if (records.sizeInBytes == 0)
       throw new IllegalArgumentException("Attempt to append an empty record 
set")
 
-    val appendInfo = log.appendAsFollower(records.asInstanceOf[MemoryRecords])
-    new LogAppendInfo(appendInfo.firstOffset.getOrElse {
-      throw new KafkaException("Append failed unexpectedly")
-    }, appendInfo.lastOffset)
+    
handleAndConvertLogAppendInfo(log.appendAsFollower(records.asInstanceOf[MemoryRecords]))
+  }
+
+  private def handleAndConvertLogAppendInfo(appendInfo: 
kafka.log.LogAppendInfo): LogAppendInfo = {
+    appendInfo.firstOffset match {
+      case Some(firstOffset) =>
+        if (firstOffset.relativePositionInSegment == 0) {
+          // Assume that a new segment was created if the relative position is 0
+          log.deleteOldSegments()
+        }
+        new LogAppendInfo(firstOffset.messageOffset, appendInfo.lastOffset)
+      case None =>
+        throw new KafkaException(s"Append failed unexpectedly: $appendInfo")
+    }
   }
 
   override def lastFetchedEpoch: Int = {
-    log.latestEpoch.getOrElse(0)
+    log.latestEpoch.getOrElse {
+      latestSnapshotId.map { snapshotId =>
+        val logEndOffset = endOffset().offset
+        if (snapshotId.offset == startOffset && snapshotId.offset == 
logEndOffset) {
+          // Return the epoch of the snapshot when the log is empty
+          snapshotId.epoch
+        } else {
+          throw new KafkaException(
+            s"Log doesn't have a last fetch epoch and there is a snapshot 
($snapshotId). " +
+            s"Expected the snapshot's end offset to match the log's end offset 
($logEndOffset) " +
+            s"and the log start offset ($startOffset)"
+          )
+        }
+      }.orElse(0)
+    }
   }
 
   override def endOffsetForEpoch(leaderEpoch: Int): 
Optional[raft.OffsetAndEpoch] = {
     val endOffsetOpt = log.endOffsetForEpoch(leaderEpoch).map { offsetAndEpoch 
=>
-      new raft.OffsetAndEpoch(offsetAndEpoch.offset, 
offsetAndEpoch.leaderEpoch)
+      if (oldestSnapshotId.isPresent() &&
+        offsetAndEpoch.offset == oldestSnapshotId.get().offset &&
+        offsetAndEpoch.leaderEpoch == leaderEpoch) {

Review comment:
       I'm ok with the suggestion about `validateFetchOffsetAndEpoch`. If we do 
that, my only ask is that we document at this level the cases that are possible 
and how they should be handled by the implementation. I don't think there are 
so many of them. Basically what happens when last fetched epoch/offset is out 
of range.




----------------------------------------------------------------
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:
[email protected]


Reply via email to