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/09/09 21:28:09 UTC

[GitHub] [kafka] hachikuji opened a new pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

hachikuji opened a new pull request #9275:
URL: https://github.com/apache/kafka/pull/9275


   This patch bumps the `Fetch` protocol as specified by KIP-595: https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum. The main differences are the following:
   
   - Truncation detection 
   - Leader discovery through the response
   - Flexible version support
   
   The most notable change is truncation detection. This patch adds logic in the request handling path to detect truncation, but it does not change the replica fetchers to make use of this capability. We are planning to do this separately.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r488833309



##########
File path: core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
##########
@@ -47,63 +47,81 @@ class LeaderEpochFileCache(topicPartition: TopicPartition,
   private val epochs = new util.TreeMap[Int, EpochEntry]()
 
   inWriteLock(lock) {
-    checkpoint.read().foreach { entry =>
-      epochs.put(entry.epoch, entry)
-    }
+    checkpoint.read().foreach(assign)
   }
 
   /**
     * Assigns the supplied Leader Epoch to the supplied Offset
     * Once the epoch is assigned it cannot be reassigned
     */
   def assign(epoch: Int, startOffset: Long): Unit = {
-    inWriteLock(lock) {
-      val updateNeeded = if (epochs.isEmpty) {
-        true
-      } else {
-        val lastEntry = epochs.lastEntry.getValue
-        lastEntry.epoch != epoch || startOffset < lastEntry.startOffset
-      }
+    val entry = EpochEntry(epoch, startOffset)
+    if (assign(entry)) {
+      debug(s"Appended new epoch entry $entry. Cache now contains ${epochs.size} entries.")
+      flush()
+    }
+  }
 
-      if (updateNeeded) {
-        truncateAndAppend(EpochEntry(epoch, startOffset))
-        flush()
+  private def assign(entry: EpochEntry): Boolean = {
+    if (entry.epoch < 0 || entry.startOffset < 0) {
+      throw new IllegalArgumentException(s"Received invalid partition leader epoch entry $entry")
+    }
+
+    // Check whether the append is needed before acquiring the write lock
+    // in order to avoid contention with readers in the common case
+    latestEntry.foreach { lastEntry =>
+      if (entry.epoch == lastEntry.epoch && entry.startOffset >= lastEntry.startOffset) {
+        return false

Review comment:
       Do we need to check again inside the lock before update because this is outside the lock? 




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



[GitHub] [kafka] hachikuji commented on pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#issuecomment-690583324


   Note this patch includes #9277. I will rebase once it is merged.


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



[GitHub] [kafka] hachikuji commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r488850209



##########
File path: core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
##########
@@ -47,63 +47,81 @@ class LeaderEpochFileCache(topicPartition: TopicPartition,
   private val epochs = new util.TreeMap[Int, EpochEntry]()
 
   inWriteLock(lock) {
-    checkpoint.read().foreach { entry =>
-      epochs.put(entry.epoch, entry)
-    }
+    checkpoint.read().foreach(assign)
   }
 
   /**
     * Assigns the supplied Leader Epoch to the supplied Offset
     * Once the epoch is assigned it cannot be reassigned
     */
   def assign(epoch: Int, startOffset: Long): Unit = {
-    inWriteLock(lock) {
-      val updateNeeded = if (epochs.isEmpty) {
-        true
-      } else {
-        val lastEntry = epochs.lastEntry.getValue
-        lastEntry.epoch != epoch || startOffset < lastEntry.startOffset
-      }
+    val entry = EpochEntry(epoch, startOffset)
+    if (assign(entry)) {
+      debug(s"Appended new epoch entry $entry. Cache now contains ${epochs.size} entries.")
+      flush()
+    }
+  }
 
-      if (updateNeeded) {
-        truncateAndAppend(EpochEntry(epoch, startOffset))
-        flush()
+  private def assign(entry: EpochEntry): Boolean = {
+    if (entry.epoch < 0 || entry.startOffset < 0) {
+      throw new IllegalArgumentException(s"Received invalid partition leader epoch entry $entry")
+    }
+
+    // Check whether the append is needed before acquiring the write lock
+    // in order to avoid contention with readers in the common case
+    latestEntry.foreach { lastEntry =>
+      if (entry.epoch == lastEntry.epoch && entry.startOffset >= lastEntry.startOffset) {
+        return false

Review comment:
       Yeah, let me add it. I had debated it and convinced myself it was not needed, but that probably relies a bit too heavily on locking at higher layers. Probably better to err on the safe side.




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



[GitHub] [kafka] ijuma commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r489612109



##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -100,7 +100,9 @@ object ApiVersion {
     // Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
     KAFKA_2_6_IV0,
     // Introduced feature versioning support (KIP-584)
-    KAFKA_2_7_IV0
+    KAFKA_2_7_IV0,
+    // Bup Fetch protocol for Raft protocol (KIP-595)

Review comment:
       Maybe you can fix this typo in your next PR.




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



[GitHub] [kafka] hachikuji merged pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji merged pull request #9275:
URL: https://github.com/apache/kafka/pull/9275


   


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



[GitHub] [kafka] hachikuji commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r488309917



##########
File path: core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
##########
@@ -91,8 +95,23 @@ class LeaderEpochFileCache(topicPartition: TopicPartition,
     }
   }
 
+  def removeEntries(predicate: EpochEntry => Boolean): Seq[EpochEntry] = {
+    val removedEpochs = mutable.ListBuffer.empty[EpochEntry]
+    val iterator = epochs.entrySet().iterator()
+
+    while (iterator.hasNext) {

Review comment:
       That's a good question. We probably could be smarter in `truncateAndAppend` if it was safe to assume epoch/start offset increases monotonically. Maybe we just need to add the logic to validate that invariant on startup. Let me take a look at that.




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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r487974401



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -232,19 +263,25 @@ public FetchRequest build(short version) {
             // We collect the partitions in a single FetchTopic only if they appear sequentially in the fetchData
             FetchRequestData.FetchTopic fetchTopic = null;
             for (Map.Entry<TopicPartition, PartitionData> entry : fetchData.entrySet()) {
-                if (fetchTopic == null || !entry.getKey().topic().equals(fetchTopic.topic())) {
+                TopicPartition topicPartition = entry.getKey();
+                PartitionData partitionData = entry.getValue();
+
+                if (fetchTopic == null || !topicPartition.topic().equals(fetchTopic.topic())) {
                     fetchTopic = new FetchRequestData.FetchTopic()
-                       .setTopic(entry.getKey().topic())
+                       .setTopic(topicPartition.topic())
                        .setPartitions(new ArrayList<>());
                     fetchRequestData.topics().add(fetchTopic);
                 }
 
-                fetchTopic.partitions().add(
-                    new FetchRequestData.FetchPartition().setPartition(entry.getKey().partition())
-                        .setCurrentLeaderEpoch(entry.getValue().currentLeaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
-                        .setFetchOffset(entry.getValue().fetchOffset)
-                        .setLogStartOffset(entry.getValue().logStartOffset)
-                        .setPartitionMaxBytes(entry.getValue().maxBytes));
+                FetchRequestData.FetchPartition fetchPartition = new FetchRequestData.FetchPartition()
+                    .setPartition(topicPartition.partition())
+                    .setCurrentLeaderEpoch(partitionData.currentLeaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
+                    .setFetchOffset(partitionData.fetchOffset)
+                    .setLogStartOffset(partitionData.logStartOffset)
+                    .setPartitionMaxBytes(partitionData.maxBytes);
+                partitionData.lastFetchedEpoch.ifPresent(fetchPartition::setLastFetchedEpoch);

Review comment:
       `currentLeaderEpoch` and `lastFetchEpoch` are both set to -1 by default, but they are set in different ways above, is that deliberate?

##########
File path: core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
##########
@@ -212,6 +212,44 @@ class FetchRequestTest extends BaseRequestTest {
     assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, partitionData.error)
   }
 
+  @Test
+  def testLastFetchedEpochValidation(): Unit = {
+    val topic = "topic"
+    val topicPartition = new TopicPartition(topic, 0)
+    val partitionToLeader = TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 3, servers)
+    val firstLeaderId = partitionToLeader(topicPartition.partition)
+    val firstLeaderEpoch = TestUtils.findLeaderEpoch(firstLeaderId, topicPartition, servers)
+
+    initProducer()
+
+    // Write some data in epoch 0
+    produceData(Seq(topicPartition), 100)
+    // Force a leader change
+    killBroker(firstLeaderId)
+    // Write some more data
+    produceData(Seq(topicPartition), 100)
+
+    val secondLeaderId = TestUtils.awaitLeaderChange(servers, topicPartition, firstLeaderId)
+    val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, topicPartition, servers)
+
+    // Build a fetch request at offset 150 with last fetched epoch 0
+    val fetchOffset = 150
+    val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    partitionMap.put(topicPartition, new FetchRequest.PartitionData(fetchOffset, 0L, 1024,
+      Optional.of(secondLeaderEpoch), Optional.of(firstLeaderEpoch)))
+    val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, partitionMap).build()
+
+    // Validate the expected truncation
+    val fetchResponse = sendFetchRequest(secondLeaderId, fetchRequest)
+    val partitionData = fetchResponse.responseData.get(topicPartition)
+    assertEquals(Errors.NONE, partitionData.error)
+    assertEquals(0L, partitionData.records.sizeInBytes())
+    assertTrue(partitionData.truncationOffset.isPresent)
+
+    // Should be exactly 100, but use a fuzzy truncation estimate in case there were produce retries
+    assertTrue(partitionData.truncationOffset.get < 150)

Review comment:
       we could produce without retries, wait for completion and then check exact value?

##########
File path: core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
##########
@@ -91,8 +95,23 @@ class LeaderEpochFileCache(topicPartition: TopicPartition,
     }
   }
 
+  def removeEntries(predicate: EpochEntry => Boolean): Seq[EpochEntry] = {
+    val removedEpochs = mutable.ListBuffer.empty[EpochEntry]
+    val iterator = epochs.entrySet().iterator()
+
+    while (iterator.hasNext) {

Review comment:
       I guess we would always be removing a range from the start or end. Are we going through the whole map and checking each one because there is no suitable API to remove multiple entries from start or end?




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



[GitHub] [kafka] hachikuji commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r488271329



##########
File path: core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
##########
@@ -212,6 +212,44 @@ class FetchRequestTest extends BaseRequestTest {
     assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, partitionData.error)
   }
 
+  @Test
+  def testLastFetchedEpochValidation(): Unit = {
+    val topic = "topic"
+    val topicPartition = new TopicPartition(topic, 0)
+    val partitionToLeader = TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 3, servers)
+    val firstLeaderId = partitionToLeader(topicPartition.partition)
+    val firstLeaderEpoch = TestUtils.findLeaderEpoch(firstLeaderId, topicPartition, servers)
+
+    initProducer()
+
+    // Write some data in epoch 0
+    produceData(Seq(topicPartition), 100)
+    // Force a leader change
+    killBroker(firstLeaderId)
+    // Write some more data
+    produceData(Seq(topicPartition), 100)
+
+    val secondLeaderId = TestUtils.awaitLeaderChange(servers, topicPartition, firstLeaderId)
+    val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, topicPartition, servers)
+
+    // Build a fetch request at offset 150 with last fetched epoch 0
+    val fetchOffset = 150
+    val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    partitionMap.put(topicPartition, new FetchRequest.PartitionData(fetchOffset, 0L, 1024,
+      Optional.of(secondLeaderEpoch), Optional.of(firstLeaderEpoch)))
+    val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, partitionMap).build()
+
+    // Validate the expected truncation
+    val fetchResponse = sendFetchRequest(secondLeaderId, fetchRequest)
+    val partitionData = fetchResponse.responseData.get(topicPartition)
+    assertEquals(Errors.NONE, partitionData.error)
+    assertEquals(0L, partitionData.records.sizeInBytes())
+    assertTrue(partitionData.truncationOffset.isPresent)
+
+    // Should be exactly 100, but use a fuzzy truncation estimate in case there were produce retries
+    assertTrue(partitionData.truncationOffset.get < 150)

Review comment:
       That's a good idea. Let me try that.




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



[GitHub] [kafka] hachikuji commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r491241346



##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -100,7 +100,9 @@ object ApiVersion {
     // Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
     KAFKA_2_6_IV0,
     // Introduced feature versioning support (KIP-584)
-    KAFKA_2_7_IV0
+    KAFKA_2_7_IV0,
+    // Bup Fetch protocol for Raft protocol (KIP-595)

Review comment:
       Oops! Will do.




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



[GitHub] [kafka] hachikuji commented on a change in pull request #9275: KAFKA-10435; Fetch protocol changes for KIP-595

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9275:
URL: https://github.com/apache/kafka/pull/9275#discussion_r488262695



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -232,19 +263,25 @@ public FetchRequest build(short version) {
             // We collect the partitions in a single FetchTopic only if they appear sequentially in the fetchData
             FetchRequestData.FetchTopic fetchTopic = null;
             for (Map.Entry<TopicPartition, PartitionData> entry : fetchData.entrySet()) {
-                if (fetchTopic == null || !entry.getKey().topic().equals(fetchTopic.topic())) {
+                TopicPartition topicPartition = entry.getKey();
+                PartitionData partitionData = entry.getValue();
+
+                if (fetchTopic == null || !topicPartition.topic().equals(fetchTopic.topic())) {
                     fetchTopic = new FetchRequestData.FetchTopic()
-                       .setTopic(entry.getKey().topic())
+                       .setTopic(topicPartition.topic())
                        .setPartitions(new ArrayList<>());
                     fetchRequestData.topics().add(fetchTopic);
                 }
 
-                fetchTopic.partitions().add(
-                    new FetchRequestData.FetchPartition().setPartition(entry.getKey().partition())
-                        .setCurrentLeaderEpoch(entry.getValue().currentLeaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
-                        .setFetchOffset(entry.getValue().fetchOffset)
-                        .setLogStartOffset(entry.getValue().logStartOffset)
-                        .setPartitionMaxBytes(entry.getValue().maxBytes));
+                FetchRequestData.FetchPartition fetchPartition = new FetchRequestData.FetchPartition()
+                    .setPartition(topicPartition.partition())
+                    .setCurrentLeaderEpoch(partitionData.currentLeaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
+                    .setFetchOffset(partitionData.fetchOffset)
+                    .setLogStartOffset(partitionData.logStartOffset)
+                    .setPartitionMaxBytes(partitionData.maxBytes);
+                partitionData.lastFetchedEpoch.ifPresent(fetchPartition::setLastFetchedEpoch);

Review comment:
       I guess I was trying to rely on the default from the schema, but I agree probably no reason to be inconsistent.




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