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/11/13 19:46:46 UTC

[GitHub] [kafka] jsancio commented on a change in pull request #9590: KAFKA-7556: KafkaConsumer.beginningOffsets does not return actual first offsets

jsancio commented on a change in pull request #9590:
URL: https://github.com/apache/kafka/pull/9590#discussion_r523177211



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1306,7 +1309,7 @@ class Log(@volatile private var _dir: File,
     // in an unclean manner within log.flush.start.offset.checkpoint.interval.ms. The chance of this happening is low.
     maybeHandleIOException(s"Exception while increasing log start offset for $topicPartition to $newLogStartOffset in dir ${dir.getParent}") {
       lock synchronized {
-        if (newLogStartOffset > highWatermark)
+        if (reason != SegmentCompaction && newLogStartOffset > highWatermark)

Review comment:
       I don't think it is safe for replicas to compact past the `highWatermark`. Should we remove this change and make sure that log compaction doesn't compact past the `highWatermark`.

##########
File path: core/src/main/scala/kafka/log/LogCleaner.scala
##########
@@ -711,6 +723,9 @@ private[log] class Cleaner(val id: Int,
           shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp,
           records = retained)
         throttler.maybeThrottle(outputBuffer.limit())
+        if (newCanUpdateBaseOffset)
+          dest.updateBaseOffset(result.minOffset())
+        newCanUpdateBaseOffset = false

Review comment:
       Can we move this code to `LogSegment::append`? Meaning shouldn't the `baseOffset` of a segment always be the `baseOffset` of the first record batch appended?
   
   Is the issue that the new clean segment is created before the log cleaner knows the first clean record (record batch). If so, why don't we delay creating of the new segment until that point?

##########
File path: core/src/main/scala/kafka/log/LogCleaner.scala
##########
@@ -538,6 +539,14 @@ private[log] class Cleaner(val id: Int,
     for (group <- groupedSegments)
       cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata)
 
+    if (baseOffset == log.logStartOffset) {

Review comment:
       Why the check? Shouldn't the log cleaner always perform the algorithm in the `if` block?




----------------------------------------------------------------
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