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



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1050,7 +1050,8 @@ class Log(@volatile private var _dir: File,
                      leaderEpoch: Int,
                      origin: AppendOrigin = AppendOrigin.Client,

Review comment:
       It would be helpful to have a test case which verifies `appendAsLeader` 
with the new append origin.

##########
File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
##########
@@ -68,7 +68,7 @@ class KafkaMetadataLog(
 
     val appendInfo = log.appendAsLeader(records.asInstanceOf[MemoryRecords],
       leaderEpoch = epoch,
-      origin = AppendOrigin.Coordinator)
+      origin = AppendOrigin.Replication)

Review comment:
       I was going to ask about adding a new `AppendOrigin`. I agree that the 
behavior should be the same as for `Replication`, but it seems like it could 
lead to confusion. Maybe we could add an `AppendOrigin.RaftLeader`. Then we can 
add a simple comment which emphasizes that the Raft leader is responsible for 
assigning offsets. That would also allow us to revert the changes in 
`LogCleanerTest`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
##########
@@ -1477,6 +1477,7 @@ private void appendBatch(
     ) {
         try {
             int epoch = state.epoch();
+            batch.data.batches().forEach(recordBatch -> 
recordBatch.setPartitionLeaderEpoch(epoch));

Review comment:
       Hmm.. Do we need this? I thought we already set leader epoch here: 
https://github.com/apache/kafka/blob/trunk/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java#L256.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1050,7 +1050,8 @@ class Log(@volatile private var _dir: File,
                      leaderEpoch: Int,
                      origin: AppendOrigin = AppendOrigin.Client,
                      interBrokerProtocolVersion: ApiVersion = 
ApiVersion.latestVersion): LogAppendInfo = {
-    append(records, origin, interBrokerProtocolVersion, assignOffsets = true, 
leaderEpoch, ignoreRecordSize = false)
+    val assignOffsets = if (origin == AppendOrigin.Replication) false else true

Review comment:
       By the way, it seems we might want to rename `assignOffsets` since we 
also rely on this flag for record validation. It's a bit on the verbose side, 
but maybe `validateMessagesAndAssignOffsets` to go along with the similarly 
named method in `LogValidator`?




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


Reply via email to