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 2021/09/16 18:19:06 UTC

[GitHub] [kafka] jolshan opened a new pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

jolshan opened a new pull request #11331:
URL: https://github.com/apache/kafka/pull/11331


   Store partitions with unresolved topic Ids in the session and try to resolve on future requests.
   
   [WIP]
   
   ### 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I guess that it does not change much in the end. I was considering this in order to be consistent with how we handle this for the consumer.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   
   I think I'm misunderstanding something here. Did you mean to say append? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -1091,18 +1089,20 @@ class AbstractFetcherThreadTest {
 
     override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
       val fetchData = mutable.Map.empty[TopicPartition, FetchRequest.PartitionData]
-      partitionMap.foreach { case (partition, state) =>
+      partitionMap.foreach { case (partition, state) => 0
+        .equals(0)

Review comment:
       I have no idea why this is here.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       I think that would for instance happen when the controller fails over to an older IBP during an upgrade. This should remove the topic ids which means that v12 will be used for the next fetch request and trigger a FETCH_SESSION_TOPIC_ID_ERROR. In this particular case, re-trying directly would be the optimal way to proceed for a follower. I wonder if they are other cases to consider here.
   
   For the consumer, it is definitely different.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
+                    sessionTopicNames = Collections.emptyMap();
                 }
-                topicIds = null;
                 Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                        // Re-add the replaced partition to the end of 'next'
+                        next.put(topicPartition, nextData);
+                        entry.setValue(nextData);
+                        replaced.add(new TopicIdPartition(prevData.topicId, topicPartition));
+                    } else if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
                         next.put(topicPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(new TopicIdPartition(nextData.topicId, topicPartition));
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(new TopicIdPartition(prevData.topicId, topicPartition));
                     // If we do not have this topic ID in the builder or the session, we can not use topic IDs.
-                    if (canUseTopicIds && !topicIds.containsKey(topicPartition.topic()) && !sessionTopicIds.containsKey(topicPartition.topic()))
+                    if (canUseTopicIds && prevData.topicId == Uuid.ZERO_UUID)

Review comment:
       Should we use `equals` instead of `==`? We use `equals` at L304 btw. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -238,11 +237,13 @@ public String toString() {
          * incremental fetch requests (see below).
          */
         private LinkedHashMap<TopicPartition, PartitionData> next;
+        private Map<Uuid, String> topicNames;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
+            this.topicNames = new HashMap<>();

Review comment:
       Thanks!




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       I thought about the same name, but I thought it was a slightly different approach --> looking up in the map where it is maybe there vs. supplying the name. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -202,29 +203,30 @@ public void testSessionless() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       I moved some back.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);

Review comment:
       ah good catch.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       To clarify -- are you referring to a case where we upgraded? ie, it started with no ID in the first request and added one in the second request?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       I could theoretically check replace in the other test that checks multiple scenarios

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       I'm not sure I follow. Did you mean the other test file? 

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       This was the case I tested when we had the bug of sending v13 for this scenario. The idea was that the session was empty and we had the correct topic ID usage, not whether forgotten partitions were added correctly. I can add a check for forgotten partitions for completeness.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       It likely had something to do with how the mock client was handling metadata. But that may have been for the older version where we checked NodeApiVersion. I can try to switch it back.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2659,6 +2661,9 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
                 autoCommitIntervalMs,
                 interceptors,
                 throwOnStableOffsetNotSupported);
+        ApiVersions apiVersions = new ApiVersions();
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Nope. Looks like another change I forgot to cleanup.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   I think I'm misunderstanding something here. Did you mean to say append? 

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   
   I think I'm misunderstanding something here. Did you mean to say append? 

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Are you referring to how we changed UNKNOWN_TOPIC_ID and INCONSISTENT_TOPIC_ID?
   
   For these cases we have testFetchInconsistentTopicId and testFetchUnknownTopicId which check that we update the metadata for a partition level error.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       These tests changed from returning a top level error to partition level error.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       What logic are we thinking? Checking that the unresolved topics are handled correctly?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1542,113 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(topicIds.get("foo"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(topicIds.get("bar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(topicIds.get("zar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicIdPartition]): Unit = {
+    val partitionsInContext = ArrayBuffer.empty[TopicIdPartition]
+    context.foreachPartition { (tp, _) =>
       partitionsInContext += tp
     }
     assertEquals(partitions, partitionsInContext.toSeq)
   }
 }
+
+object FetchSessionTest {
+  def idUsageCombinations: java.util.stream.Stream[Arguments] = {
+    Seq(
+      Arguments.of(true, true),
+      Arguments.of(true, false),
+      Arguments.of(false, true),
+      Arguments.of(false, false)
+    ).asJava.stream()
+  }
+}

Review comment:
       I can add some for the equals and hash methods in CachedPartition. What classes were you thinking of for others?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Hmm. I'm not quite sure why this would not make sense. I believe it is checking the types are correct.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I think we would want to keep the authorization error. Since it just logs a message. The UNKNOWN_TOPIC_ID error would request a metadata update which doesn't make sense when there is an authorization error.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or are you just referring to a case where we don't ever have topic IDs?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       Looks like most of these changes were done by this commit: https://github.com/apache/kafka/pull/11331/commits/32c6297adb685f1863b8c7eb85f2f0965853a9f8
   so I can remove them pretty easily.

##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -1091,18 +1089,20 @@ class AbstractFetcherThreadTest {
 
     override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
       val fetchData = mutable.Map.empty[TopicPartition, FetchRequest.PartitionData]
-      partitionMap.foreach { case (partition, state) =>
+      partitionMap.foreach { case (partition, state) => 0
+        .equals(0)

Review comment:
       I have no idea why this is here.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Not quite sure what you meant here but I added this for now:
   `context1.foreachPartition((topicIdPartition, _) => assertEquals(topicIds.get("foo"), topicIdPartition.topicId))`

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       We could do that, but then this check will be a bit more complicated. 
   `context2.foreachPartition((topicIdPartition, _) => assertEquals(topicNames.get(topicIdPartition.topicId), topicIdPartition.topic))`

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       I can think more on this. 

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       STILL TODO for Friday

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       So you are asking for a test that is checking the fetcher builds the request correctly? Is this a test for the fetcher or the builder? 

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       I can do that but it will take some time. 😅

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData, so I feel like that is sufficient unless I'm missing something.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       I thought about the same name, but I thought it was a slightly different approach --> looking up in the map where it is maybe there vs. supplying the name. 

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -202,29 +203,30 @@ public void testSessionless() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       I moved some back.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);

Review comment:
       ah good catch.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       To clarify -- are you referring to a case where we upgraded? ie, it started with no ID in the first request and added one in the second request?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       I could theoretically check replace in the other test that checks multiple scenarios

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       I'm not sure I follow. Did you mean the other test file? 

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       This was the case I tested when we had the bug of sending v13 for this scenario. The idea was that the session was empty and we had the correct topic ID usage, not whether forgotten partitions were added correctly. I can add a check for forgotten partitions for completeness.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       It likely had something to do with how the mock client was handling metadata. But that may have been for the older version where we checked NodeApiVersion. I can try to switch it back.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2659,6 +2661,9 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
                 autoCommitIntervalMs,
                 interceptors,
                 throwOnStableOffsetNotSupported);
+        ApiVersions apiVersions = new ApiVersions();
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Nope. Looks like another change I forgot to cleanup.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   I think I'm misunderstanding something here. Did you mean to say append? 

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   
   I think I'm misunderstanding something here. Did you mean to say append? 

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Are you referring to how we changed UNKNOWN_TOPIC_ID and INCONSISTENT_TOPIC_ID?
   
   For these cases we have testFetchInconsistentTopicId and testFetchUnknownTopicId which check that we update the metadata for a partition level error.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       These tests changed from returning a top level error to partition level error.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       What logic are we thinking? Checking that the unresolved topics are handled correctly?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1542,113 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(topicIds.get("foo"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(topicIds.get("bar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(topicIds.get("zar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicIdPartition]): Unit = {
+    val partitionsInContext = ArrayBuffer.empty[TopicIdPartition]
+    context.foreachPartition { (tp, _) =>
       partitionsInContext += tp
     }
     assertEquals(partitions, partitionsInContext.toSeq)
   }
 }
+
+object FetchSessionTest {
+  def idUsageCombinations: java.util.stream.Stream[Arguments] = {
+    Seq(
+      Arguments.of(true, true),
+      Arguments.of(true, false),
+      Arguments.of(false, true),
+      Arguments.of(false, false)
+    ).asJava.stream()
+  }
+}

Review comment:
       I can add some for the equals and hash methods in CachedPartition. What classes were you thinking of for others?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Hmm. I'm not quite sure why this would not make sense. I believe it is checking the types are correct.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I think we would want to keep the authorization error. Since it just logs a message. The UNKNOWN_TOPIC_ID error would request a metadata update which doesn't make sense when there is an authorization error.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or are you just referring to a case where we don't ever have topic IDs?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       Looks like most of these changes were done by this commit: https://github.com/apache/kafka/pull/11331/commits/32c6297adb685f1863b8c7eb85f2f0965853a9f8
   so I can remove them pretty easily.

##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -1091,18 +1089,20 @@ class AbstractFetcherThreadTest {
 
     override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
       val fetchData = mutable.Map.empty[TopicPartition, FetchRequest.PartitionData]
-      partitionMap.foreach { case (partition, state) =>
+      partitionMap.foreach { case (partition, state) => 0
+        .equals(0)

Review comment:
       I have no idea why this is here.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Not quite sure what you meant here but I added this for now:
   `context1.foreachPartition((topicIdPartition, _) => assertEquals(topicIds.get("foo"), topicIdPartition.topicId))`

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       We could do that, but then this check will be a bit more complicated. 
   `context2.foreachPartition((topicIdPartition, _) => assertEquals(topicNames.get(topicIdPartition.topicId), topicIdPartition.topic))`

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       I can think more on this. 

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       STILL TODO for Friday

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       So you are asking for a test that is checking the fetcher builds the request correctly? Is this a test for the fetcher or the builder? 

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       I can do that but it will take some time. 😅

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData, so I feel like that is sufficient unless I'm missing something.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       I thought about the same name, but I thought it was a slightly different approach --> looking up in the map where it is maybe there vs. supplying the name. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +272,54 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    sessionTopicNames = topicNames;
                 } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
+                    sessionTopicNames = Collections.emptyMap();
                 }
-                topicIds = null;
                 Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.

Review comment:
       nit: It might be worth expanding this comment a little more.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));

Review comment:
       nit: Could we use `assertEquals`? The advantage is that it ensure that the map contains only what we want.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);

Review comment:
       nit: In this case, we could actually do the following which seems a bit better:
   
   ```
   TopicIdPartition tp = new new TopicIdPartition(topicId, "topic", 0);
   ```
   
   Then, we can use `tp.topicPartition` when we need it. What do you think?

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());

Review comment:
       nit: Could we actually compare the content of both collections instead of only verifying their size? That would be more complete.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);

Review comment:
       I am not sure that we gain much by testing this because testing `fetchData` already verify that all the partitions are included, no?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    if (this.topic == null) {
+      this.topic = name

Review comment:
       Could we add a unit test for this?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +174,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
-
-  def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int =

Review comment:
       Could we add a unit test for this one as well?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =

Review comment:
       nit: Should we put `topicId` first to be consistent with `TopicIdPartition`'s constructor?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData2.put(foo0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(100)
+      .setLastStableOffset(100)
+      .setLogStartOffset(100))
+    respData2.put(foo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()

Review comment:
       I guess that `respData1` is used by mistake here, isn't it? This is a good example why it is better to use `assertEquals` to verify collections instead of iterating over them. The assertions that you have below have not caught this.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
##########
@@ -948,6 +950,94 @@ class ReplicaFetcherThreadTest {
     assertProcessPartitionDataWhen(isReassigning = false)
   }
 
+  @Test
+  def testBuildFetch(): Unit = {
+    val tid1p0 = new TopicIdPartition(topicId1, t1p0)
+    val tid1p1 = new TopicIdPartition(topicId1, t1p1)
+    val tid2p1 = new TopicIdPartition(topicId2, t2p1)
+
+    val props = TestUtils.createBrokerConfig(1, "localhost:1234")
+    val config = KafkaConfig.fromProps(props)
+    val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+    val mockBlockingSend: BlockingSend = createMock(classOf[BlockingSend])
+    val replicaQuota: ReplicaQuota = createNiceMock(classOf[ReplicaQuota])
+    val log: UnifiedLog = createNiceMock(classOf[UnifiedLog])
+
+    expect(replicaManager.brokerTopicStats).andReturn(mock(classOf[BrokerTopicStats]))
+    expect(replicaManager.localLogOrException(anyObject(classOf[TopicPartition]))).andReturn(log).anyTimes()
+    expect(replicaQuota.isThrottled(anyObject(classOf[TopicPartition]))).andReturn(false).anyTimes()
+    expect(log.logStartOffset).andReturn(0).anyTimes()
+    replay(log, replicaQuota, replicaManager)
+
+    val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, failedPartitions,
+      replicaManager, new Metrics(), new SystemTime(), replicaQuota, Some(mockBlockingSend))
+
+    val leaderEpoch = 1
+
+    val partitionMap = Map(
+        t1p0 -> PartitionFetchState(Some(topicId1), 150, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t2p1 -> PartitionFetchState(Some(topicId2), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+
+    val ResultWithPartitions(fetchRequestOpt, _) = thread.buildFetch(partitionMap)
+
+    assertTrue(fetchRequestOpt.isDefined)
+    val fetchRequestBuilder = fetchRequestOpt.get.fetchRequest
+
+    val partitionDataMap = partitionMap.map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}

Review comment:
       nit: A space is missing before `{` and `}` should be on a new line for blocks.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)

Review comment:
       Should we assert the content of `context2`?

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);
+
+        // Get the fetchData map from the request data.
+        Map<TopicIdPartition, FetchRequest.PartitionData> fetchData = fetchRequest.fetchData(topicNames);
+
+        // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+        // Otherwise, the topic name should be null.
+        // For earlier request versions, we expect topic names and zero Uuids.
+        assertEquals(expectedFetchData.size(), fetchData.size());
+        expectedFetchData.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+            TopicIdPartition tpKey = new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition()));
+            assertEquals(tidp.topicId(), fetchData.get(tpKey).topicId);
+        });

Review comment:
       nit: Would it be simpler to do the following?
   
   ```
   Map<TopicPartition, FetchRequest.PartitionData> expected = new LinkedHashMap<>();
   // Build the expected map based on fetchRequestUsesTopicIds.
   assertEquals(expected, fetchRequest.fetchData(topicNames));
   ```
   
   We have to use `new TopicIdPartition(topicId1, new TopicPartition(null, 0))` because https://github.com/apache/kafka/pull/11403 is not merged yet.
   
   The advantage of this way is that it test the whole Map, including the ordering.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
##########
@@ -948,6 +950,94 @@ class ReplicaFetcherThreadTest {
     assertProcessPartitionDataWhen(isReassigning = false)
   }
 
+  @Test
+  def testBuildFetch(): Unit = {
+    val tid1p0 = new TopicIdPartition(topicId1, t1p0)
+    val tid1p1 = new TopicIdPartition(topicId1, t1p1)
+    val tid2p1 = new TopicIdPartition(topicId2, t2p1)
+
+    val props = TestUtils.createBrokerConfig(1, "localhost:1234")
+    val config = KafkaConfig.fromProps(props)
+    val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+    val mockBlockingSend: BlockingSend = createMock(classOf[BlockingSend])
+    val replicaQuota: ReplicaQuota = createNiceMock(classOf[ReplicaQuota])
+    val log: UnifiedLog = createNiceMock(classOf[UnifiedLog])
+
+    expect(replicaManager.brokerTopicStats).andReturn(mock(classOf[BrokerTopicStats]))
+    expect(replicaManager.localLogOrException(anyObject(classOf[TopicPartition]))).andReturn(log).anyTimes()
+    expect(replicaQuota.isThrottled(anyObject(classOf[TopicPartition]))).andReturn(false).anyTimes()
+    expect(log.logStartOffset).andReturn(0).anyTimes()
+    replay(log, replicaQuota, replicaManager)
+
+    val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, failedPartitions,
+      replicaManager, new Metrics(), new SystemTime(), replicaQuota, Some(mockBlockingSend))
+
+    val leaderEpoch = 1
+
+    val partitionMap = Map(
+        t1p0 -> PartitionFetchState(Some(topicId1), 150, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t2p1 -> PartitionFetchState(Some(topicId2), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+
+    val ResultWithPartitions(fetchRequestOpt, _) = thread.buildFetch(partitionMap)
+
+    assertTrue(fetchRequestOpt.isDefined)
+    val fetchRequestBuilder = fetchRequestOpt.get.fetchRequest
+
+    val partitionDataMap = partitionMap.map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertMapEquals(partitionDataMap.asJava, fetchRequestBuilder.fetchData())
+    assertEquals(0, fetchRequestBuilder.replaced().size)
+    assertEquals(0, fetchRequestBuilder.removed().size)
+
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    responseData.put(tid1p0, new FetchResponseData.PartitionData())
+    responseData.put(tid1p1, new FetchResponseData.PartitionData())
+    responseData.put(tid2p1, new FetchResponseData.PartitionData())
+    val fetchResponse = FetchResponse.of(Errors.NONE, 0, 123, responseData)
+
+    thread.fetchSessionHandler.handleResponse(fetchResponse, ApiKeys.FETCH.latestVersion())
+
+    // Remove t1p0, change the ID for t2p1, and keep t1p1 the same
+    val newTopicId = Uuid.randomUuid()
+    val partitionMap2 = Map(
+      t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+      t2p1 -> PartitionFetchState(Some(newTopicId), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+    val ResultWithPartitions(fetchRequestOpt2, _) = thread.buildFetch(partitionMap2)
+
+    // Since t1p1 didn't change, we drop that one
+    val partitionDataMap2 = partitionMap2.drop(1).map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertTrue(fetchRequestOpt2.isDefined)
+    val fetchRequestBuilder2 = fetchRequestOpt2.get.fetchRequest
+    assertMapEquals(partitionDataMap2.asJava, fetchRequestBuilder2.fetchData())
+    assertEquals(1, fetchRequestBuilder2.replaced().size)
+    assertEquals(Collections.singletonList(tid2p1), fetchRequestBuilder2.replaced())
+    assertEquals(1, fetchRequestBuilder2.removed().size)

Review comment:
       ditto.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,

Review comment:
       nit: A space is missing before `10`.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Thinking a little more about this one. How about doing the following? We could define an helper method `fetchMessages` which wraps `replicaManager.fetchMessages` (takes the same arguments) and returns `Seq[(TopicIdPartition, FetchPartitionData)]`. This would avoid all these callbacks that we have here.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)

Review comment:
       Should we verify what the context contains? This is very likely the most important point to verify in this test, no?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -270,36 +273,320 @@ public void testFetchNormal() {
     }
 
     @Test
-    public void testFetchWithNoId() {
+    public void testFetchWithNoTopicId() {
         // Should work and default to using old request type.
         buildFetcher();
 
-        TopicPartition noId = new TopicPartition("noId", 0);
-        TopicIdPartition noIdPart = new TopicIdPartition(Uuid.ZERO_UUID, noId);
-        assignFromUserNoId(singleton(noId));
-        subscriptions.seek(noId, 0);
+        TopicIdPartition noId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("noId", 0));
+        assignFromUserNoId(singleton(noId.topicPartition()));
+        subscriptions.seek(noId.topicPartition(), 0);
 
-        // fetch should use request version 12
+        // Fetch should use request version 12
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fullFetchResponse(noIdPart, this.records, Errors.NONE, 100L, 0));
+        client.prepareResponse(
+            fetchRequestMatcher((short) 12, noId, 0, Optional.of(validLeaderEpoch)),
+            fullFetchResponse(noId, this.records, Errors.NONE, 100L, 0)
+        );
         consumerClient.poll(time.timer(0));
         assertTrue(fetcher.hasCompletedFetches());
 
         Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
-        assertTrue(partitionRecords.containsKey(noId));
+        assertTrue(partitionRecords.containsKey(noId.topicPartition()));
 
-        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(noId);
+        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(noId.topicPartition());
         assertEquals(3, records.size());
-        assertEquals(4L, subscriptions.position(noId).offset); // this is the next fetching position
+        assertEquals(4L, subscriptions.position(noId.topicPartition()).offset); // this is the next fetching position
         long offset = 1;
         for (ConsumerRecord<byte[], byte[]> record : records) {
             assertEquals(offset, record.offset());
             offset += 1;
         }
     }
 
+    @Test
+    public void testFetchWithTopicId() {
+        buildFetcher();
+
+        TopicIdPartition tp = new TopicIdPartition(topicId, new TopicPartition(topicName, 0));
+        assignFromUser(singleton(tp.topicPartition()));
+        subscriptions.seek(tp.topicPartition(), 0);
+
+        // Fetch should use latest version
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), tp, 0, Optional.of(validLeaderEpoch)),
+            fullFetchResponse(tp, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+
+        Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
+        assertTrue(partitionRecords.containsKey(tp.topicPartition()));
+
+        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(tp.topicPartition());
+        assertEquals(3, records.size());
+        assertEquals(4L, subscriptions.position(tp.topicPartition()).offset); // this is the next fetching position
+        long offset = 1;
+        for (ConsumerRecord<byte[], byte[]> record : records) {
+            assertEquals(offset, record.offset());
+            offset += 1;
+        }
+    }
+
+    @Test
+    public void testFetchForgetTopicIdWhenUnassigned() {
+        buildFetcher();
+
+        TopicIdPartition foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+        TopicIdPartition bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0));
+
+        // Assign foo and bar.
+        subscriptions.assignFromUser(singleton(foo.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(foo), tp -> validLeaderEpoch));
+        subscriptions.seek(foo.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(foo, new PartitionData(
+                    foo.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, foo, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Assign bar and unassign foo.
+        subscriptions.assignFromUser(singleton(bar.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(bar), tp -> validLeaderEpoch));
+        subscriptions.seek(bar.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(bar, new PartitionData(
+                    bar.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                singletonList(foo)
+            ),
+            fullFetchResponse(1, bar, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+    }
+
+    @Test
+    public void testFetchForgetTopicIdWhenReplaced() {
+        buildFetcher();
+
+        TopicIdPartition fooWithOldTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+        TopicIdPartition fooWithNewTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+
+        // Assign foo with old topic id.
+        subscriptions.assignFromUser(singleton(fooWithOldTopicId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithOldTopicId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithOldTopicId.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(fooWithOldTopicId, new PartitionData(
+                    fooWithOldTopicId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, fooWithOldTopicId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Replace foo with old topic id with foo with new topic id.
+        subscriptions.assignFromUser(singleton(fooWithNewTopicId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithNewTopicId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithNewTopicId.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        // foo with old topic id should be removed from the session.
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(fooWithNewTopicId, new PartitionData(
+                    fooWithNewTopicId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                singletonList(fooWithOldTopicId)
+            ),
+            fullFetchResponse(1, fooWithNewTopicId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+    }
+
+    @Test
+    public void testFetchTopicIdUpgradeDowngrade() {
+        buildFetcher();
+
+        TopicIdPartition fooWithoutId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("foo", 0));
+        TopicIdPartition fooWithId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+
+        // Assign foo without a topic id.
+        subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithoutId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithoutId.topicPartition(), 0);
+
+        // Fetch should use version 12.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher((short) 12,
+                singletonMap(fooWithoutId, new PartitionData(
+                    fooWithoutId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, fooWithoutId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Upgrade.
+        subscriptions.assignFromUser(singleton(fooWithId.topicPartition()));

Review comment:
       No, it is not. I kept it for completeness.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // Likely if the topic ID is not in the broker, the partition has been deleted. In this case return UNKNOWN_TOPIC_OR_PARTITION
+    // If we started with unknown IDs and switched to having them, we won't have an error.
+    val errorCode2 = if (endsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = if (endsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse2 = if (endsWithTopicIds) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)
+
+    assertEquals(Errors.NONE, resp2.error)
     assertTrue(resp2.sessionId > 0)
     val responseData2 = resp2.responseData(topicNames, request2.version)
-    // We should have no partition responses with this top level error.
-    assertEquals(0, responseData2.size())
+    if (startsWithTopicIds && endsWithTopicIds) {
+      // if both requests had topic IDs there was no change so we won't have a response
+      assertEquals(0, responseData2.size())
+    } else {
+      assertEquals(errorCode2, responseData2.get(tp0).errorCode)
+    }
+  }
+
+  // This test simulates trying to forget a topic partition with all possible topic ID usages for both requests.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testToForgetCases(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request forgetting the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, Collections.singletonList(tidp0), false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    // If we forget the last partition, we will have a sessionless context.
+    assertEquals(classOf[SessionlessFetchContext], context2.getClass)

Review comment:
       Should we do another round before this one to ensure that a partition would be removed from the context while still having an `IncrementalFetchContext`? We could perhaps have multiple partitions in the context, resolved and unresolved, and then we could remove them one by one.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);
+
+        // Get the fetchData map from the request data.
+        Map<TopicIdPartition, FetchRequest.PartitionData> fetchData = fetchRequest.fetchData(topicNames);
+
+        // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+        // Otherwise, the topic name should be null.
+        // For earlier request versions, we expect topic names and zero Uuids.
+        assertEquals(expectedFetchData.size(), fetchData.size());
+        expectedFetchData.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+            TopicIdPartition tpKey = new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition()));

Review comment:
       This is a bit weird. I would have expected a `null` as the topic name if `topicNames` does not contain the mapping, no?

##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -185,6 +183,53 @@ class AbstractFetcherThreadTest {
       "secondWorkDuration: " + secondWorkDuration + " was not greater than or equal to fetchBackOffMs: " + fetchBackOffMs)
   }
 
+  @Test
+  def testPartitionsInError(): Unit = {
+    val partition1 = new TopicPartition("topic1", 0)
+    val partition2 = new TopicPartition("topic2", 0)
+    val partition3 = new TopicPartition("topic3", 0)
+    val fetchBackOffMs = 250
+
+    class ErrorMockFetcherThread(fetchBackOffMs: Int)
+      extends MockFetcherThread(fetchBackOffMs =  fetchBackOffMs) {
+
+      override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = {
+        Map(partition1 -> new FetchData().setErrorCode(Errors.UNKNOWN_TOPIC_ID.code),
+            partition2 -> new FetchData().setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code),
+            partition3 -> new FetchData().setErrorCode(Errors.NONE.code))
+      }
+    }
+    val fetcher = new ErrorMockFetcherThread(fetchBackOffMs = fetchBackOffMs)

Review comment:
       nit: fyi, you could use an anonymous class in this case:
   ```
   val fetcher = new MockFetcherThread(fetchBackOffMs = fetchBackOffMs) {
     override def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = {
     
     }
   }

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topic == null)
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       As discussed, we must test this.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,

Review comment:
       ditto.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
##########
@@ -948,6 +950,94 @@ class ReplicaFetcherThreadTest {
     assertProcessPartitionDataWhen(isReassigning = false)
   }
 
+  @Test
+  def testBuildFetch(): Unit = {
+    val tid1p0 = new TopicIdPartition(topicId1, t1p0)
+    val tid1p1 = new TopicIdPartition(topicId1, t1p1)
+    val tid2p1 = new TopicIdPartition(topicId2, t2p1)
+
+    val props = TestUtils.createBrokerConfig(1, "localhost:1234")
+    val config = KafkaConfig.fromProps(props)
+    val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+    val mockBlockingSend: BlockingSend = createMock(classOf[BlockingSend])
+    val replicaQuota: ReplicaQuota = createNiceMock(classOf[ReplicaQuota])
+    val log: UnifiedLog = createNiceMock(classOf[UnifiedLog])
+
+    expect(replicaManager.brokerTopicStats).andReturn(mock(classOf[BrokerTopicStats]))
+    expect(replicaManager.localLogOrException(anyObject(classOf[TopicPartition]))).andReturn(log).anyTimes()
+    expect(replicaQuota.isThrottled(anyObject(classOf[TopicPartition]))).andReturn(false).anyTimes()
+    expect(log.logStartOffset).andReturn(0).anyTimes()
+    replay(log, replicaQuota, replicaManager)
+
+    val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, failedPartitions,
+      replicaManager, new Metrics(), new SystemTime(), replicaQuota, Some(mockBlockingSend))
+
+    val leaderEpoch = 1
+
+    val partitionMap = Map(
+        t1p0 -> PartitionFetchState(Some(topicId1), 150, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t2p1 -> PartitionFetchState(Some(topicId2), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+
+    val ResultWithPartitions(fetchRequestOpt, _) = thread.buildFetch(partitionMap)
+
+    assertTrue(fetchRequestOpt.isDefined)
+    val fetchRequestBuilder = fetchRequestOpt.get.fetchRequest
+
+    val partitionDataMap = partitionMap.map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertMapEquals(partitionDataMap.asJava, fetchRequestBuilder.fetchData())
+    assertEquals(0, fetchRequestBuilder.replaced().size)
+    assertEquals(0, fetchRequestBuilder.removed().size)
+
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    responseData.put(tid1p0, new FetchResponseData.PartitionData())
+    responseData.put(tid1p1, new FetchResponseData.PartitionData())
+    responseData.put(tid2p1, new FetchResponseData.PartitionData())
+    val fetchResponse = FetchResponse.of(Errors.NONE, 0, 123, responseData)
+
+    thread.fetchSessionHandler.handleResponse(fetchResponse, ApiKeys.FETCH.latestVersion())
+
+    // Remove t1p0, change the ID for t2p1, and keep t1p1 the same
+    val newTopicId = Uuid.randomUuid()
+    val partitionMap2 = Map(
+      t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+      t2p1 -> PartitionFetchState(Some(newTopicId), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+    val ResultWithPartitions(fetchRequestOpt2, _) = thread.buildFetch(partitionMap2)
+
+    // Since t1p1 didn't change, we drop that one
+    val partitionDataMap2 = partitionMap2.drop(1).map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertTrue(fetchRequestOpt2.isDefined)
+    val fetchRequestBuilder2 = fetchRequestOpt2.get.fetchRequest
+    assertMapEquals(partitionDataMap2.asJava, fetchRequestBuilder2.fetchData())

Review comment:
       Can't we use `assertEquals` here?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames

Review comment:
       Should we pass `topicNamesForRequest1` instead of `topicNames` here? In practice, we already use the same mapping in all cases when the context is created.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);
+
+        // Get the fetchData map from the request data.
+        Map<TopicIdPartition, FetchRequest.PartitionData> fetchData = fetchRequest.fetchData(topicNames);
+
+        // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+        // Otherwise, the topic name should be null.
+        // For earlier request versions, we expect topic names and zero Uuids.
+        assertEquals(expectedFetchData.size(), fetchData.size());
+        expectedFetchData.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+            TopicIdPartition tpKey = new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition()));
+            assertEquals(tidp.topicId(), fetchData.get(tpKey).topicId);
+        });
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testForgottenTopics(short version) {
+        // Forgotten topics are not allowed prior to version 7
+        if (version >= 7) {
+            TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+            TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+            Uuid topicId0 = Uuid.randomUuid();
+            Uuid topicId1 = Uuid.randomUuid();
+            // Only include topic IDs for the first topic partition.
+            Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+
+            // Include one topic with topic IDs in the topic names map and one without.
+            List<TopicIdPartition> toForgetTopics = new LinkedList<>();
+            toForgetTopics.add(new TopicIdPartition(topicId0, topicPartition0));
+            toForgetTopics.add(new TopicIdPartition(topicId1, topicPartition1));
+
+            boolean fetchRequestUsesTopicIds = version >= 13;
+
+            FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                    .forReplica(version, 0, 1, 1, Collections.emptyMap())
+                    .removed(toForgetTopics)
+                    .replaced(Collections.emptyList())
+                    .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+            // For versions < 13, we will be provided a topic name and a zero Uuid in FetchRequestData.
+            // Versions 13+ will contain a valid topic ID but an empty topic name.
+            List<TopicIdPartition> expectedForgottenTopicData = new LinkedList<>();
+            toForgetTopics.forEach(tidp -> {
+                String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+                Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+                expectedForgottenTopicData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+            });
+
+            // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+            List<TopicIdPartition> convertedForgottenTopicData = new LinkedList<>();
+            fetchRequest.data().forgottenTopicsData().forEach(forgottenTopic ->
+                    forgottenTopic.partitions().forEach(partition ->
+                            convertedForgottenTopicData.add(new TopicIdPartition(forgottenTopic.topic(), forgottenTopic.topicId(), partition))
+                    )
+            );
+            // The TopicIdPartitions built from the request data should match what we expect.
+            assertListEquals(expectedForgottenTopicData, convertedForgottenTopicData);
+
+            // Get the forgottenTopics from the request data.
+            List<TopicIdPartition> forgottenTopics = fetchRequest.forgottenTopics(topicNames);
+
+            // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+            // Otherwise, the topic name should be null.
+            // For earlier request versions, we expect topic names and zero Uuids.
+            // Build the list of expected TopicIdPartitions. These are different from the earlier expected topicIdPartitions
+            // as empty strings are converted to nulls.
+            assertEquals(expectedForgottenTopicData.size(), forgottenTopics.size());
+            List<TopicIdPartition> expectedForgottenTopics = new LinkedList<>();
+            expectedForgottenTopicData.forEach(tidp -> {
+                String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+                expectedForgottenTopics.add(new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition())));
+            });
+            assertListEquals(expectedForgottenTopics, forgottenTopics);

Review comment:
       Can't we use `assertEquals`? It seems that it should work here.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());

Review comment:
       nit: Could we use `assertEquals` here as well?

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
##########
@@ -948,6 +950,94 @@ class ReplicaFetcherThreadTest {
     assertProcessPartitionDataWhen(isReassigning = false)
   }
 
+  @Test
+  def testBuildFetch(): Unit = {
+    val tid1p0 = new TopicIdPartition(topicId1, t1p0)
+    val tid1p1 = new TopicIdPartition(topicId1, t1p1)
+    val tid2p1 = new TopicIdPartition(topicId2, t2p1)
+
+    val props = TestUtils.createBrokerConfig(1, "localhost:1234")
+    val config = KafkaConfig.fromProps(props)
+    val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+    val mockBlockingSend: BlockingSend = createMock(classOf[BlockingSend])
+    val replicaQuota: ReplicaQuota = createNiceMock(classOf[ReplicaQuota])
+    val log: UnifiedLog = createNiceMock(classOf[UnifiedLog])
+
+    expect(replicaManager.brokerTopicStats).andReturn(mock(classOf[BrokerTopicStats]))
+    expect(replicaManager.localLogOrException(anyObject(classOf[TopicPartition]))).andReturn(log).anyTimes()
+    expect(replicaQuota.isThrottled(anyObject(classOf[TopicPartition]))).andReturn(false).anyTimes()
+    expect(log.logStartOffset).andReturn(0).anyTimes()
+    replay(log, replicaQuota, replicaManager)
+
+    val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, failedPartitions,
+      replicaManager, new Metrics(), new SystemTime(), replicaQuota, Some(mockBlockingSend))
+
+    val leaderEpoch = 1
+
+    val partitionMap = Map(
+        t1p0 -> PartitionFetchState(Some(topicId1), 150, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t2p1 -> PartitionFetchState(Some(topicId2), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+
+    val ResultWithPartitions(fetchRequestOpt, _) = thread.buildFetch(partitionMap)
+
+    assertTrue(fetchRequestOpt.isDefined)
+    val fetchRequestBuilder = fetchRequestOpt.get.fetchRequest
+
+    val partitionDataMap = partitionMap.map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertMapEquals(partitionDataMap.asJava, fetchRequestBuilder.fetchData())
+    assertEquals(0, fetchRequestBuilder.replaced().size)
+    assertEquals(0, fetchRequestBuilder.removed().size)
+
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    responseData.put(tid1p0, new FetchResponseData.PartitionData())
+    responseData.put(tid1p1, new FetchResponseData.PartitionData())
+    responseData.put(tid2p1, new FetchResponseData.PartitionData())
+    val fetchResponse = FetchResponse.of(Errors.NONE, 0, 123, responseData)
+
+    thread.fetchSessionHandler.handleResponse(fetchResponse, ApiKeys.FETCH.latestVersion())
+
+    // Remove t1p0, change the ID for t2p1, and keep t1p1 the same
+    val newTopicId = Uuid.randomUuid()
+    val partitionMap2 = Map(
+      t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+      t2p1 -> PartitionFetchState(Some(newTopicId), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+    val ResultWithPartitions(fetchRequestOpt2, _) = thread.buildFetch(partitionMap2)
+
+    // Since t1p1 didn't change, we drop that one
+    val partitionDataMap2 = partitionMap2.drop(1).map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertTrue(fetchRequestOpt2.isDefined)
+    val fetchRequestBuilder2 = fetchRequestOpt2.get.fetchRequest
+    assertMapEquals(partitionDataMap2.asJava, fetchRequestBuilder2.fetchData())
+    assertEquals(1, fetchRequestBuilder2.replaced().size)

Review comment:
       The size is implicitly verified by the next assertion. We could remove it, I guess.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
##########
@@ -948,6 +950,94 @@ class ReplicaFetcherThreadTest {
     assertProcessPartitionDataWhen(isReassigning = false)
   }
 
+  @Test
+  def testBuildFetch(): Unit = {
+    val tid1p0 = new TopicIdPartition(topicId1, t1p0)
+    val tid1p1 = new TopicIdPartition(topicId1, t1p1)
+    val tid2p1 = new TopicIdPartition(topicId2, t2p1)
+
+    val props = TestUtils.createBrokerConfig(1, "localhost:1234")
+    val config = KafkaConfig.fromProps(props)
+    val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+    val mockBlockingSend: BlockingSend = createMock(classOf[BlockingSend])
+    val replicaQuota: ReplicaQuota = createNiceMock(classOf[ReplicaQuota])
+    val log: UnifiedLog = createNiceMock(classOf[UnifiedLog])
+
+    expect(replicaManager.brokerTopicStats).andReturn(mock(classOf[BrokerTopicStats]))
+    expect(replicaManager.localLogOrException(anyObject(classOf[TopicPartition]))).andReturn(log).anyTimes()
+    expect(replicaQuota.isThrottled(anyObject(classOf[TopicPartition]))).andReturn(false).anyTimes()
+    expect(log.logStartOffset).andReturn(0).anyTimes()
+    replay(log, replicaQuota, replicaManager)
+
+    val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, failedPartitions,
+      replicaManager, new Metrics(), new SystemTime(), replicaQuota, Some(mockBlockingSend))
+
+    val leaderEpoch = 1
+
+    val partitionMap = Map(
+        t1p0 -> PartitionFetchState(Some(topicId1), 150, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+        t2p1 -> PartitionFetchState(Some(topicId2), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+
+    val ResultWithPartitions(fetchRequestOpt, _) = thread.buildFetch(partitionMap)
+
+    assertTrue(fetchRequestOpt.isDefined)
+    val fetchRequestBuilder = fetchRequestOpt.get.fetchRequest
+
+    val partitionDataMap = partitionMap.map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}
+
+    assertMapEquals(partitionDataMap.asJava, fetchRequestBuilder.fetchData())
+    assertEquals(0, fetchRequestBuilder.replaced().size)
+    assertEquals(0, fetchRequestBuilder.removed().size)
+
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    responseData.put(tid1p0, new FetchResponseData.PartitionData())
+    responseData.put(tid1p1, new FetchResponseData.PartitionData())
+    responseData.put(tid2p1, new FetchResponseData.PartitionData())
+    val fetchResponse = FetchResponse.of(Errors.NONE, 0, 123, responseData)
+
+    thread.fetchSessionHandler.handleResponse(fetchResponse, ApiKeys.FETCH.latestVersion())
+
+    // Remove t1p0, change the ID for t2p1, and keep t1p1 the same
+    val newTopicId = Uuid.randomUuid()
+    val partitionMap2 = Map(
+      t1p1 -> PartitionFetchState(Some(topicId1), 155, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None),
+      t2p1 -> PartitionFetchState(Some(newTopicId), 160, None, leaderEpoch, None, state = Fetching, lastFetchedEpoch = None))
+    val ResultWithPartitions(fetchRequestOpt2, _) = thread.buildFetch(partitionMap2)
+
+    // Since t1p1 didn't change, we drop that one
+    val partitionDataMap2 = partitionMap2.drop(1).map{ case (tp, state) =>
+      (tp, new FetchRequest.PartitionData(state.topicId.get, state.fetchOffset, 0L,
+        config.replicaFetchMaxBytes, Optional.of(state.currentLeaderEpoch), Optional.empty()))}

Review comment:
       ditto about the code format.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;

Review comment:
       nit: Could we put this on the top of the test?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {

Review comment:
       Should we make it private?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {

Review comment:
       Should we make it private?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData2.put(foo0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(100)
+      .setLastStableOffset(100)
+      .setLogStartOffset(100))
+    respData2.put(foo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    val resp2 = context2.updateAndGenerateResponseData(respData2)
+    // Since we are ignoring IDs, we should have no errors.
+    assertEquals(Errors.NONE, resp2.error())
+    assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp2.responseData(topicNames, request2.version).size)
+    resp2.responseData(topicNames, request2.version).forEach( (tp, resp) =>
+      if (tp.topic.equals("foo"))
+       assertEquals(Errors.NONE.code, resp.errorCode)
+      else
+       assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))

Review comment:
       nit: I am not a fan of this. I would usually prefer something like the following in this case. I guess that it is a matter of taste so I leave it up to you.
   
   ```
   assertEquals(
     Map(
       "foo" -> Errors.NONE.code,
       // others
     ),
     resp2.responseData(topicNames, request2.version).asScala.map { case (tp, resp) =>
       tp -> resp.errorCode
     }
   )
   ```

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)), data2.sessionPartitions());
+            // The topicNames map will have the new topic ID if it is valid.
+            // The old topic ID should be removed as the map will be empty if the request doesn't use topic IDs.
+            assertEquals(endsWithTopicIds, handler.sessionTopicNames().containsKey(topicId2));
+            assertFalse(handler.sessionTopicNames().containsKey(topicId1));

Review comment:
       nit: I wonder if doing the following would be a bit more complete?
   
   ```
   if (endsWithTopicIds) {
     assertEquals(singletonMap(topicId2.topicId, topicId2.name), handler.sessionTopicNames());
   } else {
     assertEquals(emptyMap(), handler.sessionTopicNames());
   }
   ```

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))

Review comment:
       nit: There is an extra space after `forEach(`.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {
+    if (this.topic == null) {
+      this.topic = topicNames.get(this.topicId)

Review comment:
       Could we add a unit test for this new method?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {

Review comment:
       `startsWithTopicIds` and `endsWithTopicIds` are a bit misleading here. I suppose that they refer to either the broker knows about the topic id or not (present in its metadata cache). Am I right?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)

Review comment:
       I find those block of code really hard to read. I wonder if we could simplify them.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Correct. The method would return the full response. Then we can assert it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       Is this replacing `testUpdatedPartitionResolvesId`? This is definitely cleaner, but I'm not sure we are covering the same cases here. For context, the test I mentioned before is testing different update scenarios (I probably named it poorly). Mostly the idea is that the update method works correctly. (ie, we update a partition that once had a topic ID to one that does not, etc). Maybe that is covered in some of the other tests I've added (like `def maybeUpdateRequestParamsOrName`) and we can just remove that test. What do you think? I'll also think about this a bit more.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   Thanks for the PR. A high-level question, what are we trying to optimize for here?
   1. Requests that don't include topic ids
   2. Requests that include topic ids
   3. Both
   4. Some kind of balance of both where we compromise a bit to keep the code maintainable


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -378,53 +378,47 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
   * @param reqMetadata        The request metadata.
   * @param fetchData          The partition data from the fetch request.
   * @param usesTopicIds       True if this session should use topic IDs.
-  * @param topicIds           The map from topic names to topic IDs.
   * @param isFromFollower     True if this fetch request came from a follower.
   */
 class FullFetchContext(private val time: Time,
                        private val cache: FetchSessionCache,
                        private val reqMetadata: JFetchMetadata,
-                       private val fetchData: util.Map[TopicPartition, FetchRequest.PartitionData],
+                       private val fetchData: util.Map[TopicIdPartition, FetchRequest.PartitionData],
                        private val usesTopicIds: Boolean,
-                       private val topicIds: util.Map[String, Uuid],
                        private val isFromFollower: Boolean) extends FetchContext {
-  override def getFetchOffset(part: TopicPartition): Option[Long] =
+  override def getFetchOffset(part: TopicIdPartition): Option[Long] =
     Option(fetchData.get(part)).map(_.fetchOffset)
 
-  override def foreachPartition(fun: (TopicPartition, Uuid, FetchRequest.PartitionData) => Unit): Unit = {
-    fetchData.forEach((tp, data) => fun(tp, topicIds.get(tp.topic), data))
+  override def foreachPartition(fun: (TopicIdPartition, FetchRequest.PartitionData) => Unit): Unit = {
+    fetchData.forEach((tp, data) => fun(tp, data))
   }
 
   override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = {
-    FetchResponse.sizeOf(versionId, updates.entrySet.iterator, topicIds)
+    FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
   }
 
   override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
     var hasInconsistentTopicIds = false
-    def createNewSession: (FetchSession.CACHE_MAP, FetchSession.TOPIC_ID_MAP) = {
+    def createNewSession: FetchSession.CACHE_MAP = {
       val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
-      val sessionTopicIds = new util.HashMap[String, Uuid](updates.size)
       updates.forEach { (part, respData) =>
         if (respData.errorCode() == Errors.INCONSISTENT_TOPIC_ID.code()) {
           info(s"Session encountered an inconsistent topic ID for topicPartition $part.")
           hasInconsistentTopicIds = true
         }
         val reqData = fetchData.get(part)
-        val id = topicIds.getOrDefault(part.topic(), Uuid.ZERO_UUID)
-        cachedPartitions.mustAdd(new CachedPartition(part, id, reqData, respData))
-        if (id != Uuid.ZERO_UUID)
-          sessionTopicIds.put(part.topic, id)
+        cachedPartitions.mustAdd(new CachedPartition(part.topicPartition, part.topicId, reqData, respData))
       }
-      (cachedPartitions, sessionTopicIds)
+      cachedPartitions
     }
     val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower,
         updates.size, usesTopicIds, () => createNewSession)
     if (hasInconsistentTopicIds) {
-      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP, Collections.emptyMap())
+      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP)

Review comment:
       Yeah. We can change this but the issue was with how we deal with this partition after the error is returned. With the changes to the FetchSessionHandler, we will be able to distinguish the topics, but the implementation I have now still delays partitions on a topic partition level. We don't want to delay the topic partition with the valid ID though! There may be something we can do to handle this case.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -328,22 +329,18 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(fetchTopic.topicId());
                         }
-                        if (name != null) {
-                            // If topic name is resolved, simply add to fetchData map
-                            fetchTopic.partitions().forEach(fetchPartition ->
-                                    fetchData.put(new TopicPartition(name, fetchPartition.partition()),
-                                            new PartitionData(
-                                                    fetchPartition.fetchOffset(),
-                                                    fetchPartition.logStartOffset(),
-                                                    fetchPartition.partitionMaxBytes(),
-                                                    optionalEpoch(fetchPartition.currentLeaderEpoch()),
-                                                    optionalEpoch(fetchPartition.lastFetchedEpoch())
-                                            )
-                                    )
-                            );
-                        } else {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", fetchTopic.topicId()));
-                        }
+                        // If topic name is resolved, simply add to fetchData map

Review comment:
       nit: remove comment -- we add unresolved names now too.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Actually, you're right. That is not entirely true. I thought that the `requireNonNull` for the `topic` in one of the [constructor](https://github.com/apache/kafka/pull/11374/files#diff-3d6aa1dec2a2548f28148717926536cc937acec2ab4bd03a7bcdc58c84a6cbbaR38) would prevent this to work. However as we use the other `TopicIdPartition` constructor in this case, it is not impacted by the `requireNonNull`.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -378,53 +378,47 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
   * @param reqMetadata        The request metadata.
   * @param fetchData          The partition data from the fetch request.
   * @param usesTopicIds       True if this session should use topic IDs.
-  * @param topicIds           The map from topic names to topic IDs.
   * @param isFromFollower     True if this fetch request came from a follower.
   */
 class FullFetchContext(private val time: Time,
                        private val cache: FetchSessionCache,
                        private val reqMetadata: JFetchMetadata,
-                       private val fetchData: util.Map[TopicPartition, FetchRequest.PartitionData],
+                       private val fetchData: util.Map[TopicIdPartition, FetchRequest.PartitionData],
                        private val usesTopicIds: Boolean,
-                       private val topicIds: util.Map[String, Uuid],
                        private val isFromFollower: Boolean) extends FetchContext {
-  override def getFetchOffset(part: TopicPartition): Option[Long] =
+  override def getFetchOffset(part: TopicIdPartition): Option[Long] =
     Option(fetchData.get(part)).map(_.fetchOffset)
 
-  override def foreachPartition(fun: (TopicPartition, Uuid, FetchRequest.PartitionData) => Unit): Unit = {
-    fetchData.forEach((tp, data) => fun(tp, topicIds.get(tp.topic), data))
+  override def foreachPartition(fun: (TopicIdPartition, FetchRequest.PartitionData) => Unit): Unit = {
+    fetchData.forEach((tp, data) => fun(tp, data))
   }
 
   override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = {
-    FetchResponse.sizeOf(versionId, updates.entrySet.iterator, topicIds)
+    FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
   }
 
   override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
     var hasInconsistentTopicIds = false
-    def createNewSession: (FetchSession.CACHE_MAP, FetchSession.TOPIC_ID_MAP) = {
+    def createNewSession: FetchSession.CACHE_MAP = {
       val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
-      val sessionTopicIds = new util.HashMap[String, Uuid](updates.size)
       updates.forEach { (part, respData) =>
         if (respData.errorCode() == Errors.INCONSISTENT_TOPIC_ID.code()) {
           info(s"Session encountered an inconsistent topic ID for topicPartition $part.")
           hasInconsistentTopicIds = true
         }
         val reqData = fetchData.get(part)
-        val id = topicIds.getOrDefault(part.topic(), Uuid.ZERO_UUID)
-        cachedPartitions.mustAdd(new CachedPartition(part, id, reqData, respData))
-        if (id != Uuid.ZERO_UUID)
-          sessionTopicIds.put(part.topic, id)
+        cachedPartitions.mustAdd(new CachedPartition(part.topicPartition, part.topicId, reqData, respData))
       }
-      (cachedPartitions, sessionTopicIds)
+      cachedPartitions
     }
     val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower,
         updates.size, usesTopicIds, () => createNewSession)
     if (hasInconsistentTopicIds) {
-      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP, Collections.emptyMap())
+      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP)

Review comment:
       ^ this is still something we need to resolve.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   TODOs:
   1. Change inconsistent topic ID so it is no longer a top level error
   2. Maybe refactor some of the receiving side code, we have a map with TopicIdPartition, PartitionData and both contain topic ID


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       Could we iterate over `sessionPartitions` and directly populate `sessionTopicNames` by using `putIfAbsent` or even `put`? The grouping seems unnecessary to me here unless I am missing something.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));

Review comment:
       As `toSend` is not used before L288, how about putting this line over there?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
             }
 
             if (log.isDebugEnabled()) {
-                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {} " +
-                          "out of {}", nextMetadata, node, partitionsToLogString(added),
-                          partitionsToLogString(altered), partitionsToLogString(removed),
-                          partitionsToLogString(sessionPartitions.keySet()));
+                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {}, " +
+                                "replaced {} out of {}", nextMetadata, node, topicIdPartitionsToLogString(added),

Review comment:
       nit: Could we align like it was before?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,

Review comment:
       Same comment as before.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchMetadata.java
##########
@@ -120,6 +120,13 @@ public FetchMetadata nextCloseExisting() {
         return new FetchMetadata(sessionId, INITIAL_EPOCH);
     }
 
+    /**
+     * Return the metadata for the next closed session response.
+     */
+    public FetchMetadata closeExisting() {

Review comment:
       It seems that this method is not used anymore. Could we remove it?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       `that.canEqual(this)` seems weird to me. It seems that we could just remove it.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)

Review comment:
       nit: We could add another constructor which takes a `TopicIdPartition`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       nit: There is an extra space after `== null`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +864,15 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
-        response.data().responses().forEach { topicResponse =>
-          topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+        response.data.responses.forEach { topicResponse =>
+          topicResponse.partitions.forEach { data =>
+            // If the topic name was not known, we will have no bytes out.
+            if (topicResponse.topic != null) {
+              val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, data.partitionIndex()))

Review comment:
       nit: Parenthesis after `partitionIndex` could be omitted.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3497,14 +3493,13 @@ object KafkaApis {
   // TODO: remove resolvedResponseData method when sizeOf can take a data object.
   private[server] def sizeOfThrottledPartitions(versionId: Short,
                                                 unconvertedResponse: FetchResponse,
-                                                quota: ReplicationQuotaManager,
-                                                topicIds: util.Map[String, Uuid]): Int = {
-    val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+                                                quota: ReplicationQuotaManager): Int = {
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     unconvertedResponse.data.responses().forEach(topicResponse =>
       topicResponse.partitions().forEach(partition =>
-        responseData.put(new TopicPartition(topicResponse.topic(), partition.partitionIndex()), partition)))
+        responseData.put(new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), partition.partitionIndex())), partition)))

Review comment:
       nit: Parenthesis after partitionIndex could be omitted.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1186,7 +1174,7 @@ class ReplicaManager(val config: KafkaConfig,
             lastStableOffset = None,
             exception = Some(e))
         case e: Throwable =>
-          brokerTopicStats.topicStats(tp.topic).failedFetchRequestRate.mark()
+          brokerTopicStats.topicStats(tp.topicPartition.topic).failedFetchRequestRate.mark()

Review comment:
       nit: `tp.topic`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
         }
       }
     } else {
       // Regular Kafka consumers need READ permission on each partition they are fetching.
-      val partitionDatas = new mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]
-      fetchContext.foreachPartition { (topicPartition, topicId, partitionData) =>
-        partitionDatas += topicPartition -> partitionData
-        sessionTopicIds.put(topicPartition.topic(), topicId)
-      }
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
-      partitionDatas.foreach { case (topicPartition, data) =>
-        if (!authorizedTopics.contains(topicPartition.topic))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
-        else if (!metadataCache.contains(topicPartition))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
+      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
+        if (topicIdPartition.topicPartition.topic == null)
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+        else
+          partitionDatas += topicIdPartition -> partitionData
+      }
+      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
+      partitionDatas.foreach { case (topicIdPartition, data) =>
+        if (!authorizedTopics.contains(topicIdPartition.topicPartition.topic))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        else if (!metadataCache.contains(topicIdPartition.topicPartition))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
         else
-          interesting += (topicPartition -> data)
+          interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/DelayedFetch.scala
##########
@@ -92,7 +92,7 @@ class DelayedFetch(delayMs: Long,
         val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val partition = replicaManager.getPartitionOrException(topicPartition)
+            val partition = replicaManager.getPartitionOrException(topicPartition.topicPartition)

Review comment:
       Yeah, that would be great. `topicPartition.topicPartition` looks really weird while reading.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode

Review comment:
       nit: Should we format the code as follow?
   
   ```
   override def hashCode: Int = {
     if (topicId != Uuid.ZERO_UUID)
       (31 * partition) + topicId.hashCode
     else
       (31 * partition) + topic.hashCode
   }
   ```

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Is this method still used? I can't find any usages of it.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -364,10 +405,7 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(forgottenTopic.topicId());
                         }
-                        if (name == null) {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", forgottenTopic.topicId()));
-                        }
-                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicPartition(name, partitionId)));
+                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId))));

Review comment:
       I would also add a small comment here.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -328,31 +374,26 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(fetchTopic.topicId());
                         }
-                        if (name != null) {
-                            // If topic name is resolved, simply add to fetchData map
-                            fetchTopic.partitions().forEach(fetchPartition ->
-                                    fetchData.put(new TopicPartition(name, fetchPartition.partition()),
-                                            new PartitionData(
-                                                    fetchPartition.fetchOffset(),
-                                                    fetchPartition.logStartOffset(),
-                                                    fetchPartition.partitionMaxBytes(),
-                                                    optionalEpoch(fetchPartition.currentLeaderEpoch()),
-                                                    optionalEpoch(fetchPartition.lastFetchedEpoch())
-                                            )
-                                    )
-                            );
-                        } else {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", fetchTopic.topicId()));
-                        }
+                        fetchTopic.partitions().forEach(fetchPartition ->
+                                fetchData.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())),

Review comment:
       Should we add a comment here which explains that the topic name might be null in `TopicIdPartition` if we were unable to resolve it?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -432,9 +425,9 @@ private String partitionsToLogString(Collection<TopicPartition> partitions) {
     String verifyFullFetchResponsePartitions(Set<TopicPartition> topicPartitions, Set<Uuid> ids, short version) {
         StringBuilder bld = new StringBuilder();
         Set<TopicPartition> extra =
-            findMissing(topicPartitions, sessionPartitions.keySet());
+                findMissing(topicPartitions, sessionPartitions.keySet());

Review comment:
       nit: This change and the following ones do not seem necessary. I would revert them back.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Putting this here but it is not related to this line.
   
   It seems that we have an opportunity in `processFetchRequest` to better handle the `FETCH_SESSION_TOPIC_ID_ERROR` error. At the moment, it delays all the partitions. It seems to me that we could retry directly, no? If you agree, we could file a Jira and address this in a subsequent PR.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +222,49 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            removed.forEach(topicIdPartition -> {
+                FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                if (forgottenTopic == null) {
+                    forgottenTopic = new ForgottenTopic()
+                        .setTopic(topicIdPartition.topic())
+                        .setTopicId(topicIdPartition.topicId());
+                    forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                }
+                forgottenTopic.partitions().add(topicIdPartition.partition());
+            });
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                replaced.forEach(topicIdPartition -> {
+                    FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                    if (forgottenTopic == null) {
+                        forgottenTopic = new ForgottenTopic()
+                            .setTopic(topicIdPartition.topic())
+                            .setTopicId(topicIdPartition.topicId());
+                        forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                    }
+                    forgottenTopic.partitions().add(topicIdPartition.partition());
+                });

Review comment:
       This block is identical to the previous one. Should we pull it into a helper method? (yeah, I know, I wrote this...)

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();

Review comment:
       Not related to this PR but could we use `Collections.emtpyMap` here? That would avoid allocating a `HashMap` all the times.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I wonder if we should reply with `UNKNOWN_TOPIC_ID` for the topics whose are not resolved.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)

Review comment:
       nit: The if/else inline reads a bit weird. Should we extract the if/else?
   
   ```
   this.eq(that) || if (this.topicId != Uuid.ZERO_UUID)
     this.partition.equals(that.partition) && this.topicId.equals(that.topicId)
   else
     this.partition.equals(that.partition) && this.topic.equals(that.topic)
   ```

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)

Review comment:
       nit: It might be better to encapsulate this in `CachedPartition`. We could add a method called `maybeSetTopicName` or piggy back on `updateRequestParams`. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -801,23 +795,23 @@ class KafkaApis(val requestChannel: RequestChannel,
                 // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
                 // client.
                 new FetchResponseData.PartitionData()
-                  .setPartitionIndex(tp.partition)
+                  .setPartitionIndex(tp.topicPartition.partition)

Review comment:
       nit: We can use `tp.partition` here and a few other places.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)

Review comment:
       nit: How about naming it `cachedPartitionKey`? We could also benefits from passing `TopicIdPartition` to the constructor directly. 

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {

Review comment:
       Is `usesTopicIds` used anywhere in this method?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -224,10 +224,8 @@ class ReplicaFetcherThread(name: String,
     }
     val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
     if (!fetchSessionHandler.handleResponse(fetchResponse, clientResponse.requestHeader().apiVersion())) {
-      // If we had a topic ID related error, throw it, otherwise return an empty fetch data map.
-      if (fetchResponse.error == Errors.UNKNOWN_TOPIC_ID ||
-          fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR ||
-          fetchResponse.error == Errors.INCONSISTENT_TOPIC_ID) {
+      // If we had a session topic ID related error, throw it, otherwise return an empty fetch data map.
+      if (fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR) {

Review comment:
       I already mentioned this before but it seems that we could retry immediately in this case when the session was upgraded/downgraded. That would avoid having to wait for the backoff.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       nit: `topicIdPartition.topic` should work.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Do we still use this constructor?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1041,26 +1038,26 @@ class ReplicaManager(val config: KafkaConfig,
     //                        5) we found a diverging epoch
     if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || hasDivergingEpoch) {
       val fetchPartitionData = logReadResults.map { case (tp, result) =>
-        val isReassignmentFetch = isFromFollower && isAddingReplica(tp, replicaId)
+        val isReassignmentFetch = isFromFollower && isAddingReplica(tp.topicPartition, replicaId)
         tp -> result.toFetchPartitionData(isReassignmentFetch)
       }
       responseCallback(fetchPartitionData)
     } else {
       // construct the fetch results from the read results
-      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicPartition, FetchPartitionStatus)]
-      fetchInfos.foreach { case (topicPartition, partitionData) =>
-        logReadResultMap.get(topicPartition).foreach(logReadResult => {
+      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
+      fetchInfos.foreach { case (topicIdPartition, partitionData) =>
+        logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
           val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
-          fetchPartitionStatus += (topicPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
+          fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
       val fetchMetadata: SFetchMetadata = SFetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit,
-        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, topicIds, fetchPartitionStatus)
+        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, fetchPartitionStatus)
       val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, clientMetadata,
         responseCallback)
 
       // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
+      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp.topicPartition) }

Review comment:
       nit: We could add another `apply` method to `TopicPartitionOperationKey` which accepts a `TopicIdPartition`. That will be convenient.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)), data2.sessionPartitions());
+            // The topicNames map will have the new topic ID if it is valid.
+            // The old topic ID should be removed as the map will be empty if the request doesn't use topic IDs.
+            assertEquals(endsWithTopicIds, handler.sessionTopicNames().containsKey(topicId2));
+            assertFalse(handler.sessionTopicNames().containsKey(topicId1));

Review comment:
       Are we thinking this would be in the if block? Or in a separate one outside?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);

Review comment:
       I was mostly testing the serialization here, but maybe that's not important? I can remove if we don't need 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);
+
+        // Get the fetchData map from the request data.
+        Map<TopicIdPartition, FetchRequest.PartitionData> fetchData = fetchRequest.fetchData(topicNames);
+
+        // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+        // Otherwise, the topic name should be null.
+        // For earlier request versions, we expect topic names and zero Uuids.
+        assertEquals(expectedFetchData.size(), fetchData.size());
+        expectedFetchData.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+            TopicIdPartition tpKey = new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition()));

Review comment:
       the expectedName will be null if it is not in the map. map.get returns null if the ID is not in the map.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Right. You might have to assert on the request in the fetcher as well. As you said, we can't really get the data out from the builder otherwise.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Hmm, so this looks like another case of not having a test file for the java (unit test version) I can create that and add the tests you've been mentioning here. Alternatively I can put the tests in the scala integration test file. Seems like there are unit tests mixed in there too.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Sorry I'm still a bit confused. The request is sent in this method. We don't get access to the request. We have access to the data that is tested in FetchSessionHandler and that is passed into this method where the request is built and sent. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       This was the case I tested when we had the bug of sending v13 for this scenario. The idea was that the session was empty and we had the correct topic ID usage, not whether forgotten partitions were added correctly. I can add a check for forgotten partitions for completeness.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
+                    sessionTopicNames = Collections.emptyMap();
                 }
-                topicIds = null;
                 Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                        // Re-add the replaced partition to the end of 'next'
+                        next.put(topicPartition, nextData);
+                        entry.setValue(nextData);
+                        replaced.add(new TopicIdPartition(prevData.topicId, topicPartition));
+                    } else if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
                         next.put(topicPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(new TopicIdPartition(nextData.topicId, topicPartition));
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(new TopicIdPartition(prevData.topicId, topicPartition));
                     // If we do not have this topic ID in the builder or the session, we can not use topic IDs.
-                    if (canUseTopicIds && !topicIds.containsKey(topicPartition.topic()) && !sessionTopicIds.containsKey(topicPartition.topic()))
+                    if (canUseTopicIds && prevData.topicId == Uuid.ZERO_UUID)

Review comment:
       I think for correctness either works, but I will switch to equals for consistency.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Are you referring to how we changed UNKNOWN_TOPIC_ID and INCONSISTENT_TOPIC_ID?
   
   For these cases we have testFetchInconsistentTopicId and testFetchUnknownTopicId which check that we update the metadata for a partition level error.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -242,65 +244,68 @@ public void testIncrementals() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            TopicPartition foo0 = new TopicPartition("foo", 0);
+            TopicPartition foo1 = new TopicPartition("foo", 1);
+            builder.add(foo0, new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+            builder.add(foo1, new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
-                    new ReqEntry("foo", 1, 10, 110, 210)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200),
+                    new ReqEntry("foo", fooId, 1, 10, 110, 210)),
                     data.toSend(), data.sessionPartitions());
             assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
             assertEquals(INITIAL_EPOCH, data.metadata().epoch());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20),
-                        new RespEntry("foo", 1, 10, 20)), topicIds);
+                respMap(new RespEntry("foo", 0, fooId, 10, 20),
+                        new RespEntry("foo", 1, fooId, 10, 20)));
             handler.handleResponse(resp, version);
 
             // Test an incremental fetch request which adds one partition and modifies another.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
             addTopicId(topicIds, topicNames, "bar", version);
-            builder2.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder2.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 120, 210, Optional.empty()));
-            builder2.add(new TopicPartition("bar", 0), topicIds.getOrDefault("bar", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(20, 200, 200, Optional.empty()));
+            Uuid barId = topicIds.getOrDefault("bar", Uuid.ZERO_UUID);
+            TopicPartition bar0 = new TopicPartition("bar", 0);
+            builder2.add(foo0,
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       There are a few more cases where we could put the partition data back on the previous line in this file.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -413,8 +413,20 @@ abstract class AbstractFetcherThread(name: String,
 
                 case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
                   warn(s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
-                       "This error may be returned transiently when the partition is being created or deleted, but it is not " +
-                       "expected to persist.")
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_TOPIC_ID =>
+                  warn(s"Received ${Errors.UNKNOWN_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.INCONSISTENT_TOPIC_ID =>
+                  warn(s"Received ${Errors.INCONSISTENT_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")

Review comment:
       Do we have unit tests covering those cases? There are almost no changes in `AbstractFetcherThreadTest` so it seems that we don't. Are they somewhere else perhaps?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Sorry, I wanted to say happen.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   @jolshan It seems that there are a few compilation errors, at least for `JDK 8 and Scala 2.12`. Could you check?


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       I wrote that test to illustrate how we could improve the readability. My concern is that they are so many lines/assertions in `testUpdatedPartitionResolvesId` and `testToForgetCases` that we get distracted and we have almost missed the most important assertions - the ones which validate what the session contains (`assertPartitionsOrder`). `assertPartitionsOrder` is actually the piece which ensures that the names are resolved or not, right?
   
   




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)

Review comment:
       I've concluded that your new test will now cover the necessary cases (especially with your new commit) so I think we can just remove this. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {
+    if (this.topic == null) {
+      this.topic = topicNames.get(this.topicId)

Review comment:
       We can do that. I believe this is being tested via `testFetchSessionWithUnknownId` already. What should this unit test do? Simply create a few cached partitions and do all possible operations?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    if (this.topic == null) {
+      this.topic = name

Review comment:
       I can add another one that is more explicitly testing this method, but it is tested via `testUpdatedPartitionResolvesId`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // Likely if the topic ID is not in the broker, the partition has been deleted. In this case return UNKNOWN_TOPIC_OR_PARTITION
+    // If we started with unknown IDs and switched to having them, we won't have an error.
+    val errorCode2 = if (endsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = if (endsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse2 = if (endsWithTopicIds) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)
+
+    assertEquals(Errors.NONE, resp2.error)
     assertTrue(resp2.sessionId > 0)
     val responseData2 = resp2.responseData(topicNames, request2.version)
-    // We should have no partition responses with this top level error.
-    assertEquals(0, responseData2.size())
+    if (startsWithTopicIds && endsWithTopicIds) {
+      // if both requests had topic IDs there was no change so we won't have a response
+      assertEquals(0, responseData2.size())
+    } else {
+      assertEquals(errorCode2, responseData2.get(tp0).errorCode)
+    }
+  }
+
+  // This test simulates trying to forget a topic partition with all possible topic ID usages for both requests.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testToForgetCases(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request forgetting the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, Collections.singletonList(tidp0), false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    // If we forget the last partition, we will have a sessionless context.
+    assertEquals(classOf[SessionlessFetchContext], context2.getClass)

Review comment:
       ^ this is what I've done.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,38 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {
+    if (this.topic == null) {
+      this.topic = topicNames.get(this.topicId)

Review comment:
       We can do that. I believe this is being tested via `testFetchSessionWithUnknownId` already. But an explicit test will be good.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1542,113 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(topicIds.get("foo"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(topicIds.get("bar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(topicIds.get("zar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicIdPartition]): Unit = {
+    val partitionsInContext = ArrayBuffer.empty[TopicIdPartition]
+    context.foreachPartition { (tp, _) =>
       partitionsInContext += tp
     }
     assertEquals(partitions, partitionsInContext.toSeq)
   }
 }
+
+object FetchSessionTest {
+  def idUsageCombinations: java.util.stream.Stream[Arguments] = {
+    Seq(
+      Arguments.of(true, true),
+      Arguments.of(true, false),
+      Arguments.of(false, true),
+      Arguments.of(false, false)
+    ).asJava.stream()
+  }
+}

Review comment:
       I wonder if we could add a few more unit tests. For instance, we should test the equals/hash methods of the CachedPartition (and possibly other methods there). We might want to add some for other classes as well. What do you think?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       I have simplified the code and removed a few Maps along the way. Here is the diff: https://github.com/apache/kafka/compare/trunk...dajac:KAFKA-13111.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       @jolshan With Ismael's PR (https://github.com/apache/kafka/pull/11374), this trick does not work any more. We need to think about an alternative/better approach. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Actually, you're right. That is not entirely true. I thought that the `requireNonNull` for the `topic` in one of the [constructor](https://github.com/apache/kafka/pull/11374/files#diff-3d6aa1dec2a2548f28148717926536cc937acec2ab4bd03a7bcdc58c84a6cbbaR38) would prevent this to work. However, as we use the other `TopicIdPartition` constructor here it is not impacted by the `requireNonNull`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       In this case, it would be nice if we would have a `TopicIdPartition` which contains an optional topic name. For the context, the issue is that we might have partitions in the fetch requests for which the topic name is unknown or not yet known by the broker.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Actually, you're right. That is not entirely true. I thought that the `requireNonNull` for the `topic` in one of the [constructor](https://github.com/apache/kafka/pull/11374/files#diff-3d6aa1dec2a2548f28148717926536cc937acec2ab4bd03a7bcdc58c84a6cbbaR38) would prevent this to work. However as we use the other `TopicIdPartition` constructor in this case, it is not impacted by the `requireNonNull`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Sounds good, thanks!




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,7 +183,7 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
+            (if (this.topic == null) this.topic == null && that.topic == null else this.topic.equals(that.topic)) &&

Review comment:
       can remove redundant this.topic == null




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
##########
@@ -22,16 +22,16 @@ import kafka.cluster.BrokerEndPoint
 import kafka.log.{LeaderOffsetIncremented, LogAppendInfo}
 import kafka.server.AbstractFetcherThread.{ReplicaFetch, ResultWithPartitions}
 import kafka.server.QuotaFactory.UnboundedQuota
-import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.{TopicIdPartition, TopicPartition}
 import org.apache.kafka.common.errors.KafkaStorageException
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH
 import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, RequestUtils}
-

Review comment:
       nit: fix spacing in imports -- there are a few of these in the 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -672,29 +672,22 @@ class KafkaApis(val requestChannel: RequestChannel,
     val versionId = request.header.apiVersion
     val clientId = request.header.clientId
     val fetchRequest = request.body[FetchRequest]
-    val (topicIds, topicNames) =
+    val topicNames =
       if (fetchRequest.version() >= 13)
-        metadataCache.topicIdInfo()
+        metadataCache.topicIdsToNames()
       else
-        (Collections.emptyMap[String, Uuid](), Collections.emptyMap[Uuid, String]())
+        Collections.emptyMap[Uuid, String]()
 
-    // If fetchData or forgottenTopics contain an unknown topic ID, return a top level error.
-    var fetchData: util.Map[TopicPartition, FetchRequest.PartitionData] = null
-    var forgottenTopics: util.List[TopicPartition] = null
-    try {
-      fetchData = fetchRequest.fetchData(topicNames)
-      forgottenTopics = fetchRequest.forgottenTopics(topicNames)
-    } catch {
-      case e: UnknownTopicIdException => throw e
-    }
+    val fetchData = fetchRequest.fetchData(topicNames)
+    val forgottenTopics = fetchRequest.forgottenTopics(topicNames)

Review comment:
       We could I suppose? I think the only difference is whether we pass in these values or the fetch request itself (+ topicName map). I don't know if how we handle changes based on context type (besides full/sessionless sessions not having forgotten topics). We could save time translating though if we end up having something like an error session.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       These tests changed from returning a top level error to partition level error.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       We could do that, but then this check will be a bit more complicated. 
   `context2.foreachPartition((topicIdPartition, _) => assertEquals(topicNames.get(topicIdPartition.topicId), topicIdPartition.topic))`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       I can think more on this. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Yeah, I meant exactly that. How about using `assertPartitionsOrder` helper? The assertion would be more complete.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Sorry, I meant below assertions not above. Yes, it seems that they are testing the logic of the `FetchRequest` itself and not really the logic of the FetchSessionHandler.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Correct. I was referring to the upgrade case. We might need to handle the downgrade case for https://github.com/apache/kafka/pull/11459.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       Yeah, it would be good to assert what we expect in `data2` for completeness.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Yes, I was referring to those. Ack, I missed them during my first read.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Yeah, I meant exactly that. How about using `assertPartitionsOrder` helper? The assertion would be more complete.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       You could use `assertPartitionsOrder` helper here as well.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       That is right.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Should we add or extend a test in `FetcherTest` to cover this change? I would like to have one which ensure that the request sent is populated correctly (especially the replaced part) by the fetcher based on the session handler. It seems that we don't have such test in the suite at the moment.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +235,31 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            addToForgottenTopicMap(removed, forgottenTopicMap);
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                addToForgottenTopicMap(replaced, forgottenTopicMap);
+            }

Review comment:
       Should we add a few unit tests to validate the changes that we have done in this class? We could add a few to FetchRequestTest (not use if it already exists though).

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -314,8 +356,7 @@ public int maxBytes() {
 
     // For versions < 13, builds the partitionData map using only the FetchRequestData.
     // For versions 13+, builds the partitionData map using both the FetchRequestData and a mapping of topic IDs to names.
-    // Throws UnknownTopicIdException for versions 13+ if the topic ID was unknown to the server.
-    public Map<TopicPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {
+    public Map<TopicIdPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {

Review comment:
       Do we have a unit test for this one and for `forgottenTopics`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -242,65 +244,68 @@ public void testIncrementals() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            TopicPartition foo0 = new TopicPartition("foo", 0);
+            TopicPartition foo1 = new TopicPartition("foo", 1);
+            builder.add(foo0, new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+            builder.add(foo1, new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
-                    new ReqEntry("foo", 1, 10, 110, 210)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200),
+                    new ReqEntry("foo", fooId, 1, 10, 110, 210)),
                     data.toSend(), data.sessionPartitions());
             assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
             assertEquals(INITIAL_EPOCH, data.metadata().epoch());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20),
-                        new RespEntry("foo", 1, 10, 20)), topicIds);
+                respMap(new RespEntry("foo", 0, fooId, 10, 20),
+                        new RespEntry("foo", 1, fooId, 10, 20)));
             handler.handleResponse(resp, version);
 
             // Test an incremental fetch request which adds one partition and modifies another.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
             addTopicId(topicIds, topicNames, "bar", version);
-            builder2.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder2.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 120, 210, Optional.empty()));
-            builder2.add(new TopicPartition("bar", 0), topicIds.getOrDefault("bar", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(20, 200, 200, Optional.empty()));
+            Uuid barId = topicIds.getOrDefault("bar", Uuid.ZERO_UUID);
+            TopicPartition bar0 = new TopicPartition("bar", 0);
+            builder2.add(foo0,
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       There are a few more cases where we could put the partition data back on the previous line in this file.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       I think that would for instance happen when the controller fails over to an older IBP during an upgrade. This should remove the topic ids which means that v12 will be used for the next fetch request and trigger a FETCH_SESSION_TOPIC_ID_ERROR. In this particular case, re-trying directly would be the optimal way to proceed for a follower. I wonder if they are other cases to consider here.
   
   For the consumer, it is definitely different.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Sorry, I wanted to say happen.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Anyway, we don't need to address this in this PR. I just wanted to point out that there is an opportunity for an improvement.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -413,8 +413,20 @@ abstract class AbstractFetcherThread(name: String,
 
                 case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
                   warn(s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
-                       "This error may be returned transiently when the partition is being created or deleted, but it is not " +
-                       "expected to persist.")
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_TOPIC_ID =>
+                  warn(s"Received ${Errors.UNKNOWN_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.INCONSISTENT_TOPIC_ID =>
+                  warn(s"Received ${Errors.INCONSISTENT_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")

Review comment:
       Do we have unit tests covering those cases? There are almost no changes in `AbstractFetcherThreadTest` so it seems that we don't. Are they somewhere else perhaps?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +178,37 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int =
+    if (topicId != Uuid.ZERO_UUID)
+      (31 * partition) + topicId.hashCode
+    else
+      (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]

Review comment:
       I guess that we could remove it now.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       nit: Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover them as well?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover all the methods that we have changed or added as well?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       Do we have tests verifying this change?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
+                    sessionTopicNames = Collections.emptyMap();
                 }
-                topicIds = null;
                 Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                        // Re-add the replaced partition to the end of 'next'
+                        next.put(topicPartition, nextData);
+                        entry.setValue(nextData);
+                        replaced.add(new TopicIdPartition(prevData.topicId, topicPartition));
+                    } else if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
                         next.put(topicPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(new TopicIdPartition(nextData.topicId, topicPartition));
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(new TopicIdPartition(prevData.topicId, topicPartition));
                     // If we do not have this topic ID in the builder or the session, we can not use topic IDs.
-                    if (canUseTopicIds && !topicIds.containsKey(topicPartition.topic()) && !sessionTopicIds.containsKey(topicPartition.topic()))
+                    if (canUseTopicIds && prevData.topicId == Uuid.ZERO_UUID)

Review comment:
       Should we use `equals` instead of `==`? We use `equals` at L304 btw. 

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       We should have a test in the Fetcher which ensure that the builder received the correct information. Then we could have one for the request which ensure that the builder does its job correctly as well.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Right. You might have to assert on the request in the fetcher as well. As you said, we can't really get the data out from the builder otherwise.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Yeah, I agree with you. Perhaps, we could just remove the maybeSetTopicName and move its logic into the update request params method.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Ok, so we'll pass a name and the reqData in that method.

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       So I can write a separate callback for each one that checks the ID.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       This was here before my change, but I can remove it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Hmm, how does my PR affect this?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       We should probably remove that non null check, since it's weird to have it only in that one path. I can submit a 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       @jolshan I have been looking at the changes in the `FetchSessionHandler` as well at the changes in the related classes. I am a bit worried by two things: 1) the `FetchSessionHandler` is quite complicated now, at least a bit more than before; and 2) the reliance on the request version is spread in many places now.
   
   It seems that we could get away with a simpler solution which, I think, cover all the cases as well. At the moment in the `FetchSessionHandler`, we track the `added`, `removed` and `altered` partitions and the `FetchRequest` is constructed based `next` (`added` + `altered`) and `removed`. Now imagine that we would track another list `replaced` (or `upgraded`...). We would add a partition to this list when we detect that the topic id of the partition in `next` is different from the one in the session. Then, we would pass that new list to the `FetchRequestBuilder` as well. In the builder, we would add it to the forgotten set if version >= 13 or ignore it otherwise.
   
   I have tried to implement this based on `trunk`: https://github.com/apache/kafka/commit/a1de3910ddb9b64d0890dfd61a2e8263f2aa4864. I think that we should be able to do something similar based on your version which uses `TopicIdPartition`.
   
   The pros is that the version handling remains in the `FetchRequest` class. The cons is that it does not allow to restart the session immediately without doing a round-trip to the broker, which is not a big deal as this could only happen during the upgrade.
   
   




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       Could we iterate over `sessionPartitions` and directly populate `sessionTopicNames` by using `putIfAbsent` or even `put`? The grouping seems unnecessary to me here unless I am missing something.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));

Review comment:
       As `toSend` is not used before L288, how about putting this line over there?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
             }
 
             if (log.isDebugEnabled()) {
-                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {} " +
-                          "out of {}", nextMetadata, node, partitionsToLogString(added),
-                          partitionsToLogString(altered), partitionsToLogString(removed),
-                          partitionsToLogString(sessionPartitions.keySet()));
+                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {}, " +
+                                "replaced {} out of {}", nextMetadata, node, topicIdPartitionsToLogString(added),

Review comment:
       nit: Could we align like it was before?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,

Review comment:
       Same comment as before.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchMetadata.java
##########
@@ -120,6 +120,13 @@ public FetchMetadata nextCloseExisting() {
         return new FetchMetadata(sessionId, INITIAL_EPOCH);
     }
 
+    /**
+     * Return the metadata for the next closed session response.
+     */
+    public FetchMetadata closeExisting() {

Review comment:
       It seems that this method is not used anymore. Could we remove it?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       `that.canEqual(this)` seems weird to me. It seems that we could just remove it.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)

Review comment:
       nit: We could add another constructor which takes a `TopicIdPartition`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       nit: There is an extra space after `== null`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +864,15 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
-        response.data().responses().forEach { topicResponse =>
-          topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+        response.data.responses.forEach { topicResponse =>
+          topicResponse.partitions.forEach { data =>
+            // If the topic name was not known, we will have no bytes out.
+            if (topicResponse.topic != null) {
+              val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, data.partitionIndex()))

Review comment:
       nit: Parenthesis after `partitionIndex` could be omitted.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3497,14 +3493,13 @@ object KafkaApis {
   // TODO: remove resolvedResponseData method when sizeOf can take a data object.
   private[server] def sizeOfThrottledPartitions(versionId: Short,
                                                 unconvertedResponse: FetchResponse,
-                                                quota: ReplicationQuotaManager,
-                                                topicIds: util.Map[String, Uuid]): Int = {
-    val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+                                                quota: ReplicationQuotaManager): Int = {
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     unconvertedResponse.data.responses().forEach(topicResponse =>
       topicResponse.partitions().forEach(partition =>
-        responseData.put(new TopicPartition(topicResponse.topic(), partition.partitionIndex()), partition)))
+        responseData.put(new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), partition.partitionIndex())), partition)))

Review comment:
       nit: Parenthesis after partitionIndex could be omitted.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1186,7 +1174,7 @@ class ReplicaManager(val config: KafkaConfig,
             lastStableOffset = None,
             exception = Some(e))
         case e: Throwable =>
-          brokerTopicStats.topicStats(tp.topic).failedFetchRequestRate.mark()
+          brokerTopicStats.topicStats(tp.topicPartition.topic).failedFetchRequestRate.mark()

Review comment:
       nit: `tp.topic`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
         }
       }
     } else {
       // Regular Kafka consumers need READ permission on each partition they are fetching.
-      val partitionDatas = new mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]
-      fetchContext.foreachPartition { (topicPartition, topicId, partitionData) =>
-        partitionDatas += topicPartition -> partitionData
-        sessionTopicIds.put(topicPartition.topic(), topicId)
-      }
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
-      partitionDatas.foreach { case (topicPartition, data) =>
-        if (!authorizedTopics.contains(topicPartition.topic))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
-        else if (!metadataCache.contains(topicPartition))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
+      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
+        if (topicIdPartition.topicPartition.topic == null)
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+        else
+          partitionDatas += topicIdPartition -> partitionData
+      }
+      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
+      partitionDatas.foreach { case (topicIdPartition, data) =>
+        if (!authorizedTopics.contains(topicIdPartition.topicPartition.topic))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        else if (!metadataCache.contains(topicIdPartition.topicPartition))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
         else
-          interesting += (topicPartition -> data)
+          interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/DelayedFetch.scala
##########
@@ -92,7 +92,7 @@ class DelayedFetch(delayMs: Long,
         val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val partition = replicaManager.getPartitionOrException(topicPartition)
+            val partition = replicaManager.getPartitionOrException(topicPartition.topicPartition)

Review comment:
       Yeah, that would be great. `topicPartition.topicPartition` looks really weird while reading.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode

Review comment:
       nit: Should we format the code as follow?
   
   ```
   override def hashCode: Int = {
     if (topicId != Uuid.ZERO_UUID)
       (31 * partition) + topicId.hashCode
     else
       (31 * partition) + topic.hashCode
   }
   ```

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Is this method still used? I can't find any usages of it.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -364,10 +405,7 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(forgottenTopic.topicId());
                         }
-                        if (name == null) {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", forgottenTopic.topicId()));
-                        }
-                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicPartition(name, partitionId)));
+                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId))));

Review comment:
       I would also add a small comment here.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -328,31 +374,26 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(fetchTopic.topicId());
                         }
-                        if (name != null) {
-                            // If topic name is resolved, simply add to fetchData map
-                            fetchTopic.partitions().forEach(fetchPartition ->
-                                    fetchData.put(new TopicPartition(name, fetchPartition.partition()),
-                                            new PartitionData(
-                                                    fetchPartition.fetchOffset(),
-                                                    fetchPartition.logStartOffset(),
-                                                    fetchPartition.partitionMaxBytes(),
-                                                    optionalEpoch(fetchPartition.currentLeaderEpoch()),
-                                                    optionalEpoch(fetchPartition.lastFetchedEpoch())
-                                            )
-                                    )
-                            );
-                        } else {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", fetchTopic.topicId()));
-                        }
+                        fetchTopic.partitions().forEach(fetchPartition ->
+                                fetchData.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())),

Review comment:
       Should we add a comment here which explains that the topic name might be null in `TopicIdPartition` if we were unable to resolve it?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -432,9 +425,9 @@ private String partitionsToLogString(Collection<TopicPartition> partitions) {
     String verifyFullFetchResponsePartitions(Set<TopicPartition> topicPartitions, Set<Uuid> ids, short version) {
         StringBuilder bld = new StringBuilder();
         Set<TopicPartition> extra =
-            findMissing(topicPartitions, sessionPartitions.keySet());
+                findMissing(topicPartitions, sessionPartitions.keySet());

Review comment:
       nit: This change and the following ones do not seem necessary. I would revert them back.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Putting this here but it is not related to this line.
   
   It seems that we have an opportunity in `processFetchRequest` to better handle the `FETCH_SESSION_TOPIC_ID_ERROR` error. At the moment, it delays all the partitions. It seems to me that we could retry directly, no? If you agree, we could file a Jira and address this in a subsequent PR.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +222,49 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            removed.forEach(topicIdPartition -> {
+                FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                if (forgottenTopic == null) {
+                    forgottenTopic = new ForgottenTopic()
+                        .setTopic(topicIdPartition.topic())
+                        .setTopicId(topicIdPartition.topicId());
+                    forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                }
+                forgottenTopic.partitions().add(topicIdPartition.partition());
+            });
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                replaced.forEach(topicIdPartition -> {
+                    FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                    if (forgottenTopic == null) {
+                        forgottenTopic = new ForgottenTopic()
+                            .setTopic(topicIdPartition.topic())
+                            .setTopicId(topicIdPartition.topicId());
+                        forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                    }
+                    forgottenTopic.partitions().add(topicIdPartition.partition());
+                });

Review comment:
       This block is identical to the previous one. Should we pull it into a helper method? (yeah, I know, I wrote this...)

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();

Review comment:
       Not related to this PR but could we use `Collections.emtpyMap` here? That would avoid allocating a `HashMap` all the times.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I wonder if we should reply with `UNKNOWN_TOPIC_ID` for the topics whose are not resolved.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)

Review comment:
       nit: The if/else inline reads a bit weird. Should we extract the if/else?
   
   ```
   this.eq(that) || if (this.topicId != Uuid.ZERO_UUID)
     this.partition.equals(that.partition) && this.topicId.equals(that.topicId)
   else
     this.partition.equals(that.partition) && this.topic.equals(that.topic)
   ```

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)

Review comment:
       nit: It might be better to encapsulate this in `CachedPartition`. We could add a method called `maybeSetTopicName` or piggy back on `updateRequestParams`. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -801,23 +795,23 @@ class KafkaApis(val requestChannel: RequestChannel,
                 // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
                 // client.
                 new FetchResponseData.PartitionData()
-                  .setPartitionIndex(tp.partition)
+                  .setPartitionIndex(tp.topicPartition.partition)

Review comment:
       nit: We can use `tp.partition` here and a few other places.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)

Review comment:
       nit: How about naming it `cachedPartitionKey`? We could also benefits from passing `TopicIdPartition` to the constructor directly. 

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {

Review comment:
       Is `usesTopicIds` used anywhere in this method?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -224,10 +224,8 @@ class ReplicaFetcherThread(name: String,
     }
     val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
     if (!fetchSessionHandler.handleResponse(fetchResponse, clientResponse.requestHeader().apiVersion())) {
-      // If we had a topic ID related error, throw it, otherwise return an empty fetch data map.
-      if (fetchResponse.error == Errors.UNKNOWN_TOPIC_ID ||
-          fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR ||
-          fetchResponse.error == Errors.INCONSISTENT_TOPIC_ID) {
+      // If we had a session topic ID related error, throw it, otherwise return an empty fetch data map.
+      if (fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR) {

Review comment:
       I already mentioned this before but it seems that we could retry immediately in this case when the session was upgraded/downgraded. That would avoid having to wait for the backoff.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       nit: `topicIdPartition.topic` should work.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Do we still use this constructor?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1041,26 +1038,26 @@ class ReplicaManager(val config: KafkaConfig,
     //                        5) we found a diverging epoch
     if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || hasDivergingEpoch) {
       val fetchPartitionData = logReadResults.map { case (tp, result) =>
-        val isReassignmentFetch = isFromFollower && isAddingReplica(tp, replicaId)
+        val isReassignmentFetch = isFromFollower && isAddingReplica(tp.topicPartition, replicaId)
         tp -> result.toFetchPartitionData(isReassignmentFetch)
       }
       responseCallback(fetchPartitionData)
     } else {
       // construct the fetch results from the read results
-      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicPartition, FetchPartitionStatus)]
-      fetchInfos.foreach { case (topicPartition, partitionData) =>
-        logReadResultMap.get(topicPartition).foreach(logReadResult => {
+      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
+      fetchInfos.foreach { case (topicIdPartition, partitionData) =>
+        logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
           val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
-          fetchPartitionStatus += (topicPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
+          fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
       val fetchMetadata: SFetchMetadata = SFetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit,
-        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, topicIds, fetchPartitionStatus)
+        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, fetchPartitionStatus)
       val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, clientMetadata,
         responseCallback)
 
       // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
+      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp.topicPartition) }

Review comment:
       nit: We could add another `apply` method to `TopicPartitionOperationKey` which accepts a `TopicIdPartition`. That will be convenient.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       Could we iterate over `sessionPartitions` and directly populate `sessionTopicNames` by using `putIfAbsent` or even `put`? The grouping seems unnecessary to me here unless I am missing something.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));

Review comment:
       As `toSend` is not used before L288, how about putting this line over there?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
             }
 
             if (log.isDebugEnabled()) {
-                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {} " +
-                          "out of {}", nextMetadata, node, partitionsToLogString(added),
-                          partitionsToLogString(altered), partitionsToLogString(removed),
-                          partitionsToLogString(sessionPartitions.keySet()));
+                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {}, " +
+                                "replaced {} out of {}", nextMetadata, node, topicIdPartitionsToLogString(added),

Review comment:
       nit: Could we align like it was before?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,

Review comment:
       Same comment as before.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchMetadata.java
##########
@@ -120,6 +120,13 @@ public FetchMetadata nextCloseExisting() {
         return new FetchMetadata(sessionId, INITIAL_EPOCH);
     }
 
+    /**
+     * Return the metadata for the next closed session response.
+     */
+    public FetchMetadata closeExisting() {

Review comment:
       It seems that this method is not used anymore. Could we remove it?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       `that.canEqual(this)` seems weird to me. It seems that we could just remove it.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)

Review comment:
       nit: We could add another constructor which takes a `TopicIdPartition`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       nit: There is an extra space after `== null`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +864,15 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
-        response.data().responses().forEach { topicResponse =>
-          topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+        response.data.responses.forEach { topicResponse =>
+          topicResponse.partitions.forEach { data =>
+            // If the topic name was not known, we will have no bytes out.
+            if (topicResponse.topic != null) {
+              val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, data.partitionIndex()))

Review comment:
       nit: Parenthesis after `partitionIndex` could be omitted.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3497,14 +3493,13 @@ object KafkaApis {
   // TODO: remove resolvedResponseData method when sizeOf can take a data object.
   private[server] def sizeOfThrottledPartitions(versionId: Short,
                                                 unconvertedResponse: FetchResponse,
-                                                quota: ReplicationQuotaManager,
-                                                topicIds: util.Map[String, Uuid]): Int = {
-    val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+                                                quota: ReplicationQuotaManager): Int = {
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     unconvertedResponse.data.responses().forEach(topicResponse =>
       topicResponse.partitions().forEach(partition =>
-        responseData.put(new TopicPartition(topicResponse.topic(), partition.partitionIndex()), partition)))
+        responseData.put(new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), partition.partitionIndex())), partition)))

Review comment:
       nit: Parenthesis after partitionIndex could be omitted.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1186,7 +1174,7 @@ class ReplicaManager(val config: KafkaConfig,
             lastStableOffset = None,
             exception = Some(e))
         case e: Throwable =>
-          brokerTopicStats.topicStats(tp.topic).failedFetchRequestRate.mark()
+          brokerTopicStats.topicStats(tp.topicPartition.topic).failedFetchRequestRate.mark()

Review comment:
       nit: `tp.topic`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
         }
       }
     } else {
       // Regular Kafka consumers need READ permission on each partition they are fetching.
-      val partitionDatas = new mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]
-      fetchContext.foreachPartition { (topicPartition, topicId, partitionData) =>
-        partitionDatas += topicPartition -> partitionData
-        sessionTopicIds.put(topicPartition.topic(), topicId)
-      }
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
-      partitionDatas.foreach { case (topicPartition, data) =>
-        if (!authorizedTopics.contains(topicPartition.topic))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
-        else if (!metadataCache.contains(topicPartition))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
+      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
+        if (topicIdPartition.topicPartition.topic == null)
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+        else
+          partitionDatas += topicIdPartition -> partitionData
+      }
+      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
+      partitionDatas.foreach { case (topicIdPartition, data) =>
+        if (!authorizedTopics.contains(topicIdPartition.topicPartition.topic))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        else if (!metadataCache.contains(topicIdPartition.topicPartition))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
         else
-          interesting += (topicPartition -> data)
+          interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/DelayedFetch.scala
##########
@@ -92,7 +92,7 @@ class DelayedFetch(delayMs: Long,
         val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val partition = replicaManager.getPartitionOrException(topicPartition)
+            val partition = replicaManager.getPartitionOrException(topicPartition.topicPartition)

Review comment:
       Yeah, that would be great. `topicPartition.topicPartition` looks really weird while reading.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode

Review comment:
       nit: Should we format the code as follow?
   
   ```
   override def hashCode: Int = {
     if (topicId != Uuid.ZERO_UUID)
       (31 * partition) + topicId.hashCode
     else
       (31 * partition) + topic.hashCode
   }
   ```

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Is this method still used? I can't find any usages of it.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -364,10 +405,7 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(forgottenTopic.topicId());
                         }
-                        if (name == null) {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", forgottenTopic.topicId()));
-                        }
-                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicPartition(name, partitionId)));
+                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId))));

Review comment:
       I would also add a small comment here.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -328,31 +374,26 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(fetchTopic.topicId());
                         }
-                        if (name != null) {
-                            // If topic name is resolved, simply add to fetchData map
-                            fetchTopic.partitions().forEach(fetchPartition ->
-                                    fetchData.put(new TopicPartition(name, fetchPartition.partition()),
-                                            new PartitionData(
-                                                    fetchPartition.fetchOffset(),
-                                                    fetchPartition.logStartOffset(),
-                                                    fetchPartition.partitionMaxBytes(),
-                                                    optionalEpoch(fetchPartition.currentLeaderEpoch()),
-                                                    optionalEpoch(fetchPartition.lastFetchedEpoch())
-                                            )
-                                    )
-                            );
-                        } else {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", fetchTopic.topicId()));
-                        }
+                        fetchTopic.partitions().forEach(fetchPartition ->
+                                fetchData.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())),

Review comment:
       Should we add a comment here which explains that the topic name might be null in `TopicIdPartition` if we were unable to resolve it?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -432,9 +425,9 @@ private String partitionsToLogString(Collection<TopicPartition> partitions) {
     String verifyFullFetchResponsePartitions(Set<TopicPartition> topicPartitions, Set<Uuid> ids, short version) {
         StringBuilder bld = new StringBuilder();
         Set<TopicPartition> extra =
-            findMissing(topicPartitions, sessionPartitions.keySet());
+                findMissing(topicPartitions, sessionPartitions.keySet());

Review comment:
       nit: This change and the following ones do not seem necessary. I would revert them back.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Putting this here but it is not related to this line.
   
   It seems that we have an opportunity in `processFetchRequest` to better handle the `FETCH_SESSION_TOPIC_ID_ERROR` error. At the moment, it delays all the partitions. It seems to me that we could retry directly, no? If you agree, we could file a Jira and address this in a subsequent PR.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +222,49 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            removed.forEach(topicIdPartition -> {
+                FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                if (forgottenTopic == null) {
+                    forgottenTopic = new ForgottenTopic()
+                        .setTopic(topicIdPartition.topic())
+                        .setTopicId(topicIdPartition.topicId());
+                    forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                }
+                forgottenTopic.partitions().add(topicIdPartition.partition());
+            });
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                replaced.forEach(topicIdPartition -> {
+                    FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                    if (forgottenTopic == null) {
+                        forgottenTopic = new ForgottenTopic()
+                            .setTopic(topicIdPartition.topic())
+                            .setTopicId(topicIdPartition.topicId());
+                        forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                    }
+                    forgottenTopic.partitions().add(topicIdPartition.partition());
+                });

Review comment:
       This block is identical to the previous one. Should we pull it into a helper method? (yeah, I know, I wrote this...)

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();

Review comment:
       Not related to this PR but could we use `Collections.emtpyMap` here? That would avoid allocating a `HashMap` all the times.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I wonder if we should reply with `UNKNOWN_TOPIC_ID` for the topics whose are not resolved.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)

Review comment:
       nit: The if/else inline reads a bit weird. Should we extract the if/else?
   
   ```
   this.eq(that) || if (this.topicId != Uuid.ZERO_UUID)
     this.partition.equals(that.partition) && this.topicId.equals(that.topicId)
   else
     this.partition.equals(that.partition) && this.topic.equals(that.topic)
   ```

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)

Review comment:
       nit: It might be better to encapsulate this in `CachedPartition`. We could add a method called `maybeSetTopicName` or piggy back on `updateRequestParams`. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -801,23 +795,23 @@ class KafkaApis(val requestChannel: RequestChannel,
                 // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
                 // client.
                 new FetchResponseData.PartitionData()
-                  .setPartitionIndex(tp.partition)
+                  .setPartitionIndex(tp.topicPartition.partition)

Review comment:
       nit: We can use `tp.partition` here and a few other places.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)

Review comment:
       nit: How about naming it `cachedPartitionKey`? We could also benefits from passing `TopicIdPartition` to the constructor directly. 

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {

Review comment:
       Is `usesTopicIds` used anywhere in this method?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -224,10 +224,8 @@ class ReplicaFetcherThread(name: String,
     }
     val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
     if (!fetchSessionHandler.handleResponse(fetchResponse, clientResponse.requestHeader().apiVersion())) {
-      // If we had a topic ID related error, throw it, otherwise return an empty fetch data map.
-      if (fetchResponse.error == Errors.UNKNOWN_TOPIC_ID ||
-          fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR ||
-          fetchResponse.error == Errors.INCONSISTENT_TOPIC_ID) {
+      // If we had a session topic ID related error, throw it, otherwise return an empty fetch data map.
+      if (fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR) {

Review comment:
       I already mentioned this before but it seems that we could retry immediately in this case when the session was upgraded/downgraded. That would avoid having to wait for the backoff.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       nit: `topicIdPartition.topic` should work.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Do we still use this constructor?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1041,26 +1038,26 @@ class ReplicaManager(val config: KafkaConfig,
     //                        5) we found a diverging epoch
     if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || hasDivergingEpoch) {
       val fetchPartitionData = logReadResults.map { case (tp, result) =>
-        val isReassignmentFetch = isFromFollower && isAddingReplica(tp, replicaId)
+        val isReassignmentFetch = isFromFollower && isAddingReplica(tp.topicPartition, replicaId)
         tp -> result.toFetchPartitionData(isReassignmentFetch)
       }
       responseCallback(fetchPartitionData)
     } else {
       // construct the fetch results from the read results
-      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicPartition, FetchPartitionStatus)]
-      fetchInfos.foreach { case (topicPartition, partitionData) =>
-        logReadResultMap.get(topicPartition).foreach(logReadResult => {
+      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
+      fetchInfos.foreach { case (topicIdPartition, partitionData) =>
+        logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
           val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
-          fetchPartitionStatus += (topicPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
+          fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
       val fetchMetadata: SFetchMetadata = SFetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit,
-        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, topicIds, fetchPartitionStatus)
+        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, fetchPartitionStatus)
       val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, clientMetadata,
         responseCallback)
 
       // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
+      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp.topicPartition) }

Review comment:
       nit: We could add another `apply` method to `TopicPartitionOperationKey` which accepts a `TopicIdPartition`. That will be convenient.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       STILL TODO for Friday




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -2440,6 +2442,57 @@ class KafkaApisTest {
     assertNull(partitionData.abortedTransactions)
   }
 
+  /**
+   * Verifies that partitions with unknown topic ID errors are added to the erroneous set and there is not an attempt to fetch them.
+   */
+  @Test
+  def testFetchRequestErroneousPartitions(): Unit = {
+    val tidp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val tp = tidp.topicPartition
+    val nullTp = new TopicPartition(null, tp.partition)

Review comment:
       Could we simplify all of that by defining two `TopicIdPartition`? For instance, we could have the following:
   
   ```
   val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
   val unresolvedFoo = new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition));
   ```
   
   Then, we can use them where we need them.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)

Review comment:
       I've concluded that your new test will now cover the necessary cases so I think we can just remove this. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   Looks like the topic id partition changes broke the build. I'll probably need to pull the latest version.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or are you just referring to a case where we don't ever have topic IDs?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Confirmed this was a strange quirk from 4 years ago




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       I am not sure that I follow. We should only test the FetchRequest/Builder in FetchRequestTest.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       I'm not sure I follow. Did you mean the other test file? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Correct. I was referring to the upgrade case. We might need to handle the downgrade case for https://github.com/apache/kafka/pull/11459.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Hum.. I was thinking that the method would return the partitions and we would do the assertion after. That would make the helper generic enough to be reused in other places as well. I guess that either ways would work.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       We must be able to verify that the request sent out by this method is correct. In the unit tests, we mock the network client for this purpose. If I remember correctly, we can pass a request matcher to it. I need to look into the existing unit tests for this class to see how we have done it for other cases.
   
   We might already have tests verifying that the version of the fetch request sent out is correct based on wether topic ids are used or not. If we do, I suppose that we could proceed similarly.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Would we pass in the topicIdPartition we want to match as well?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -238,11 +237,13 @@ public String toString() {
          * incremental fetch requests (see below).
          */
         private LinkedHashMap<TopicPartition, PartitionData> next;
+        private Map<Uuid, String> topicNames;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
+            this.topicNames = new HashMap<>();

Review comment:
       I added the initialization in the other builder since we were missing it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Sorry, I meant below assertions not above. Yes, it seems that they are testing the logic of the `FetchRequest` itself and not really the logic of the FetchSessionHandler.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I think we would want to keep the authorization error. Since it just logs a message. The UNKNOWN_TOPIC_ID error would request a metadata update which doesn't make sense when there is an authorization error.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala
##########
@@ -22,16 +22,16 @@ import kafka.cluster.BrokerEndPoint
 import kafka.log.{LeaderOffsetIncremented, LogAppendInfo}
 import kafka.server.AbstractFetcherThread.{ReplicaFetch, ResultWithPartitions}
 import kafka.server.QuotaFactory.UnboundedQuota
-import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.{TopicIdPartition, TopicPartition}
 import org.apache.kafka.common.errors.KafkaStorageException
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH
 import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, RequestUtils}
-

Review comment:
       nit: fix spacing in imports




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {

Review comment:
       You are correct. I can change to `startsWithTopicIdsInMetadataCache` etc if that is not too verbose.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Yeah, that works as well.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)), data2.sessionPartitions());
+            // The topicNames map will have the new topic ID if it is valid.
+            // The old topic ID should be removed as the map will be empty if the request doesn't use topic IDs.
+            assertEquals(endsWithTopicIds, handler.sessionTopicNames().containsKey(topicId2));
+            assertFalse(handler.sessionTopicNames().containsKey(topicId1));

Review comment:
       Yeah, that could remain in the if block. We could simply replaces those two lines, I guess.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Hmm. Could I also just remove the filter and do that after to use just the single callback?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   > System test results: http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2021-11-10--001.system-test-kafka-branch-builder--1636543025--jolshan--KAFKA-13111--165a106bf3/report.html
   > 
   > A previous run was all green, so will need to confirm the 3 failed tests are unrelated to this change.
   
   @jolshan Have you been able to triage these failures?


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac merged pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       Right. The question is how to validate that the first update method works? You have to get the partitions from the session as well, isn't it?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));

Review comment:
       Is `assertMapEquals` not already doing this? It seems like we check all entries and make sure there is nothing left over.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Oh so we wouldn't do the filter as part of the callback?

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,25 +793,26 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
-      }
 
-      val validFetchPartitionData = new FetchRequest.PartitionData(0L, 0L, maxFetchBytes,
+      val validFetchPartitionData = new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0L, 0L, maxFetchBytes,
         Optional.of(leaderEpoch))
 
       // Fetch messages simulating a different ID than the one in the log.
+      val inconsistentTidp = new TopicIdPartition(Uuid.randomUuid(), tidp.topicPartition)
+      def callback1(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter(_._1 == inconsistentTidp).map(_._2)
+      }

Review comment:
       Oh so we wouldn't do the filter as part of the method?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       There are two places it may be resolved -- either in the update method if the partition with the new ID is sent in the request or in the assertPartitionsOrder. I was also trying to ensure the correct error messages are returned in the response specifically via `updateAndGenerateResponseData`, but maybe we don't care about this here?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       I think we may even be able to get away with fewer maps. I see in the commit you have we add to topicIDs at the start but I'm not sure that works if we have more than one ID for a topic. I was thinking if we stored the ID in the fetch data, we wouldn't need to build a map from ids to names. Do we still use that anywhere?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       To clarify -- are you referring to a case where we upgraded? ie, it started with no ID in the first request and added one in the second request?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Or did you mean just adding something like 
   `assertEquals(fetchRequestUsesIds, data2.toReplace().size() > 0);`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       Yeah, it would be good to assert what we expect in `data2` for completeness.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover them as well?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       The idea was to not do a put operation for every partition but instead every topic. Maybe grouping is slower though.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Got it. I guess I was wondering if there would be an issue if we change semantics/expected flow for fetch again.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Seems like the other issue is that FetchSessionHandler.FetchRequestData constructor is private. So if I want to test in another file I need to either make the constructor public, create a FetchSessionHandler and duplicate the code here, or just put the values into the builder directly (skipping the class). I'm open to just putting the values directly if that makes sense.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       I think that would for instance append when the controller fails over to an older IBP during an upgrade. This should remove the topic ids which means that v12 will be used for the next fetch request and trigger a FETCH_SESSION_TOPIC_ID_ERROR. In this particular case, re-trying directly would be the optimal way to proceed for a follower. I wonder if they are other cases to consider here.
   
   For the consumer, it is definitely different.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah, that's a good question. I guess that that constructor is convenient for tests but might be bug prone in the regular code. I am tempted to remove it entirely.... What do you think?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   @jolshan It seems that there are a few compilation errors, at least for `JDK 8 and Scala 2.12`. Could you check?


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       It is curious that we don't assert the forgotten partitions here. Is there a reason?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -212,6 +217,9 @@ private void assignFromUser(Set<TopicPartition> partitions) {
         metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWithIds("dummy", 1,
             Collections.emptyMap(), singletonMap(topicName, 4),
             tp -> validLeaderEpoch, topicIds), false, 0L);
+
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Do we still need this change?

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       This is not ideal. Could we validate that the topic id is correct as well?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2659,6 +2661,9 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
                 autoCommitIntervalMs,
                 interceptors,
                 throwOnStableOffsetNotSupported);
+        ApiVersions apiVersions = new ApiVersions();
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Do we still need this change?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);

Review comment:
       Why do we use 12 here?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -202,29 +203,30 @@ public void testSessionless() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       nit: Is it worth bringing back this line on the previous one as there is space now? It might be too long though.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Would it be more appropriate to move the above assertions to `FetchRequestTest`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Should we also test when the current topic-partition in the session does not have a topic id? In this case, it should not be added to the `toReplace` set.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       Is there any reason for this change?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       The PR changed how some errors are handled in the `Fetcher`. Do we have any tests for this new behavior?

##########
File path: core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala
##########
@@ -137,11 +137,11 @@ class DelayedFetchTest extends EasyMockSupport {
 
     val fetchStatus = FetchPartitionStatus(
       startOffsetMetadata = LogOffsetMetadata(fetchOffset),
-      fetchInfo = new FetchRequest.PartitionData(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch))
+      fetchInfo = new FetchRequest.PartitionData(topicIds.get("topic"), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch))

Review comment:
       nit: It seems that we could use `TopicIdPartition` directly and remove `topicIds` map entirely. We could also pass the `TopicIdPartition` to `buildFetchMetadata`.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -465,18 +477,18 @@ public void testFetchError() {
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fullFetchResponse(tp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0));
+        client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0));
         consumerClient.poll(time.timer(0));
         assertTrue(fetcher.hasCompletedFetches());
 
         Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
         assertFalse(partitionRecords.containsKey(tp0));
     }
 
-    private MockClient.RequestMatcher matchesOffset(final TopicPartition tp, final long offset) {
+    private MockClient.RequestMatcher matchesOffset(final TopicIdPartition tp, final long offset) {
         return body -> {
             FetchRequest fetch = (FetchRequest) body;
-            Map<TopicPartition, FetchRequest.PartitionData> fetchData =  fetch.fetchData(topicNames);
+            Map<TopicIdPartition, FetchRequest.PartitionData> fetchData =  fetch.fetchData(topicNames);

Review comment:
       nit: There are two spaces after `=`.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -157,38 +156,38 @@ class FetchSessionTest {
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
 
-    val tp0 = new TopicPartition("foo", 0)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 1)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid()).asJava
+    val tp0 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 0))
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 1))
     val topicNames = topicIds.asScala.map(_.swap).asJava
 
-    def cachedLeaderEpochs(context: FetchContext): Map[TopicPartition, Optional[Integer]] = {
-      val mapBuilder = Map.newBuilder[TopicPartition, Optional[Integer]]
-      context.foreachPartition((tp, _, data) => mapBuilder += tp -> data.currentLeaderEpoch)
+    def cachedLeaderEpochs(context: FetchContext): Map[TopicIdPartition, Optional[Integer]] = {
+      val mapBuilder = Map.newBuilder[TopicIdPartition, Optional[Integer]]
+      context.foreachPartition((tp, data) => mapBuilder += tp -> data.currentLeaderEpoch)
       mapBuilder.result()
     }
 
     val requestData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    requestData1.put(tp0, new FetchRequest.PartitionData(0, 0, 100, Optional.empty()))
-    requestData1.put(tp1, new FetchRequest.PartitionData(10, 0, 100, Optional.of(1)))
-    requestData1.put(tp2, new FetchRequest.PartitionData(10, 0, 100, Optional.of(2)))
+    requestData1.put(tp0.topicPartition, new FetchRequest.PartitionData(topicIds.get("foo"), 0, 0, 100, Optional.empty()))
+    requestData1.put(tp1.topicPartition, new FetchRequest.PartitionData(topicIds.get("foo"), 10, 0, 100, Optional.of(1)))
+    requestData1.put(tp2.topicPartition, new FetchRequest.PartitionData(topicIds.get("bar"), 10, 0, 100, Optional.of(2)))

Review comment:
       nit: We could get the topic id from `tp*.topicId`.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Should we iterate over the partitions in the context to check the `TopicIdPartition`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -222,6 +230,9 @@ private void assignFromUserNoId(Set<TopicPartition> partitions) {
         metadata.update(9, RequestTestUtils.metadataUpdateWithIds("dummy", 1,
             Collections.emptyMap(), singletonMap("noId", 1),
             tp -> validLeaderEpoch, topicIds), false, 0L);
+
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       ditto. There is a few other cases in this file.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // run through each partition to resolve them
+    context2.foreachPartition((_, _) => ())

Review comment:
       Should we assert that the `TopicIdPartition` received here contains the topic name?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // run through each partition to resolve them

Review comment:
       nit: I would expand this comment a little and stress the fact that topic names are lazily resolved when the partitions are iterated over.

##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -1091,18 +1089,20 @@ class AbstractFetcherThreadTest {
 
     override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
       val fetchData = mutable.Map.empty[TopicPartition, FetchRequest.PartitionData]
-      partitionMap.foreach { case (partition, state) =>
+      partitionMap.foreach { case (partition, state) => 0
+        .equals(0)

Review comment:
       `0.equals(0)` was very likely put here by mistake.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava

Review comment:
       It seems to be that it would be simpler to declare `fooId` and `barId` and to use them instead of getting them from the map.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       Should we add any tests for the new logic in KafkaApis?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       I wonder if we should add a third topic which is never resolved. What do you think?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -526,11 +532,11 @@ class IncrementalFetchContext(private val time: Time,
           }
         }
         if (hasInconsistentTopicIds) {
-          FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, session.id, new FetchSession.RESP_MAP, Collections.emptyMap())
+          FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, session.id, new FetchSession.RESP_MAP)

Review comment:
       Since we moved an inconsistent topic ID check to the FetchResponse.of method we may want to remove this and similar checks in other contexts




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       I have simplified the code and removed a few Maps along the way. Here is the diff: https://github.com/apache/kafka/compare/trunk...dajac:KAFKA-13111.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       @jolshan I have been looking at the changes in the `FetchSessionHandler` as well at the changes in the related classes. I am a bit worried by two things: 1) the `FetchSessionHandler` is quite complicated now, at least a bit more than before; and 2) the reliance on the request version is spread in many places now.
   
   It seems that we could get away with a simpler solution which, I think, cover all the cases as well. At the moment in the `FetchSessionHandler`, we track the `added`, `removed` and `altered` partitions and the `FetchRequest` is constructed based `next` (`added` + `altered`) and `removed`. Now imagine that we would track another list `replaced` (or `upgraded`...). We would add a partition to this list when we detect that the topic id of the partition in `next` is different from the one in the session. Then, we would pass that new list to the `FetchRequestBuilder` as well. In the builder, we would add it to the forgotten set if version >= 13 or ignore it otherwise.
   
   I have tried to implement this based on `trunk`: https://github.com/apache/kafka/commit/a1de3910ddb9b64d0890dfd61a2e8263f2aa4864. I think that we should be able to do something similar based on your version which uses `TopicIdPartition`.
   
   The pros is that the version handling remains in the `FetchRequest` class. The cons is that it does not allow to restart the session immediately without doing a round-trip to the broker, which is not a big deal as this could only happen during the upgrade.
   
   What do you think? Would this approach cover all the cases?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +935,341 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNamesFooChanged
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  private def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  private def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)
+            errorResponse(Errors.UNKNOWN_TOPIC_ID.code)
+          else
+            noErrorResponse
+        )
+      }
+      context.updateAndGenerateResponseData(data).sessionId
+    }
+
+    val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0))
+    val zar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("zar", 0))
+
+    val fooUnresolved = new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition))
+    val barUnresolved = new TopicIdPartition(bar.topicId, new TopicPartition(null, bar.partition))
+    val zarUnresolved = new TopicIdPartition(zar.topicId, new TopicPartition(null, zar.partition))
+
+    // The metadata cache does not know about the topic.
+    val context1 = newContext(
+      JFetchMetadata.INITIAL,
+      Seq(foo, bar, zar),
+      Map.empty[Uuid, String]
+    )
+
+    // So the context contains unresolved partitions.
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    assertPartitionsOrder(context1, Seq(fooUnresolved, barUnresolved, zarUnresolved))
+
+    // The response is sent back to create the session.
+    val sessionId = updateAndGenerateResponseData(context1)
+
+    // The metadata cache only knows about foo.
+    val context2 = newContext(
+      new JFetchMetadata(sessionId, 1),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic)
+    )
+
+    // So foo is resolved but not the others.
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    assertPartitionsOrder(context2, Seq(foo, barUnresolved, zarUnresolved))
+
+    updateAndGenerateResponseData(context2)
+
+    // The metadata cache knows about foo and bar.
+    val context3 = newContext(
+      new JFetchMetadata(sessionId, 2),
+      Seq(bar),
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic)
+    )
+
+    // So foo and bar are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context3.getClass)
+    assertPartitionsOrder(context3, Seq(foo, bar, zarUnresolved))
+
+    updateAndGenerateResponseData(context3)
+
+    // The metadata cache knows about all topics.
+    val context4 = newContext(
+      new JFetchMetadata(sessionId, 3),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic, zar.topicId -> zar.topic)
+    )
+
+    // So all topics are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context4.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+
+    updateAndGenerateResponseData(context4)
+
+    // The metadata cache does not know about the topics anymore (e.g. deleted).
+    val context5 = newContext(
+      new JFetchMetadata(sessionId, 4),
+      Seq.empty,
+      Map.empty
+    )
+
+    // All topics remain resolved.
+    assertEquals(classOf[IncrementalFetchContext], context5.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIdsInMetadataCache: Boolean, endsWithTopicIdsInMetadataCache: Boolean): Unit = {
+    // TODO: make cleaner
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIdsInMetadataCache) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNamesForRequest1
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val partitionsInSession1 = if (startsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context1, partitionsInSession1)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIdsInMetadataCache) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIdsInMetadataCache) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIdsInMetadataCache) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNamesForRequest2
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    // We will still have the topic ID in the session if we started with topic IDs and currently do not have the ID anymore.
+    val partitionsInSession2 = if (startsWithTopicIdsInMetadataCache || endsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context2, partitionsInSession2)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // If we always had topic IDs in the metadata cache or newly resolved the ID we won't have an error.
+    // If the topic ID was once in the metadata cache and in the session, the partition has likely been deleted and would have an INCONSISTENT_TOPIC_ID error.
+    // Likely if the topic ID was never in the broker, return UNKNOWN_TOPIC_OR_PARTITION
+    val errorCode2 = if (endsWithTopicIdsInMetadataCache) Errors.NONE.code
+    else if (startsWithTopicIdsInMetadataCache) Errors.INCONSISTENT_TOPIC_ID.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = partitionsInSession2(0)
+    val fooResponse2 = if (endsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)

Review comment:
       Sure. We can remove it. I think I was concerned about the correct handling of the resolved partitions (ie, we get a response back that we can actually parse), but maybe that's not really necessary.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   @ijuma 
   > Thanks for the PR. A high-level question, what are we trying to optimize for here?
   >Requests that don't include topic ids
   Requests that include topic ids
   Both
   Some kind of balance of both where we compromise a bit to keep the code maintainable
   
   The goal of this PR is to gracefully handle the new topic case. Currently in kafka, when we create a new topic, the leader and Isr request is sent first, then the update metadata request. This means that we will often encounter transient "unknown_topic_id" errors. In the new world of topic IDs, we will see this as "unknown topic ID" errors. The current logic returns a top level error and delays all partitions. This is a regression from previous behavior, and so this PR's goal is to return to the behavior where we store the unknown partition in the session until it can be resolved. See https://issues.apache.org/jira/browse/KAFKA-13111 for more information.
   
   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       It seems like right now `elementKeysAreEqual` is just `equals`. Is the idea in implementing this to prevent someone else from doing so and not using `equals`/the logic from equals?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       Or is it that the javadoc says things like `key.elementKeysAreEqual(e) and key.hashCode() == e.hashCode()` so we should be using elementKeysAreEqual in FetchSession?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -378,53 +378,47 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
   * @param reqMetadata        The request metadata.
   * @param fetchData          The partition data from the fetch request.
   * @param usesTopicIds       True if this session should use topic IDs.
-  * @param topicIds           The map from topic names to topic IDs.
   * @param isFromFollower     True if this fetch request came from a follower.
   */
 class FullFetchContext(private val time: Time,
                        private val cache: FetchSessionCache,
                        private val reqMetadata: JFetchMetadata,
-                       private val fetchData: util.Map[TopicPartition, FetchRequest.PartitionData],
+                       private val fetchData: util.Map[TopicIdPartition, FetchRequest.PartitionData],
                        private val usesTopicIds: Boolean,
-                       private val topicIds: util.Map[String, Uuid],
                        private val isFromFollower: Boolean) extends FetchContext {
-  override def getFetchOffset(part: TopicPartition): Option[Long] =
+  override def getFetchOffset(part: TopicIdPartition): Option[Long] =
     Option(fetchData.get(part)).map(_.fetchOffset)
 
-  override def foreachPartition(fun: (TopicPartition, Uuid, FetchRequest.PartitionData) => Unit): Unit = {
-    fetchData.forEach((tp, data) => fun(tp, topicIds.get(tp.topic), data))
+  override def foreachPartition(fun: (TopicIdPartition, FetchRequest.PartitionData) => Unit): Unit = {
+    fetchData.forEach((tp, data) => fun(tp, data))
   }
 
   override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = {
-    FetchResponse.sizeOf(versionId, updates.entrySet.iterator, topicIds)
+    FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
   }
 
   override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
     var hasInconsistentTopicIds = false
-    def createNewSession: (FetchSession.CACHE_MAP, FetchSession.TOPIC_ID_MAP) = {
+    def createNewSession: FetchSession.CACHE_MAP = {
       val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
-      val sessionTopicIds = new util.HashMap[String, Uuid](updates.size)
       updates.forEach { (part, respData) =>
         if (respData.errorCode() == Errors.INCONSISTENT_TOPIC_ID.code()) {
           info(s"Session encountered an inconsistent topic ID for topicPartition $part.")
           hasInconsistentTopicIds = true
         }
         val reqData = fetchData.get(part)
-        val id = topicIds.getOrDefault(part.topic(), Uuid.ZERO_UUID)
-        cachedPartitions.mustAdd(new CachedPartition(part, id, reqData, respData))
-        if (id != Uuid.ZERO_UUID)
-          sessionTopicIds.put(part.topic, id)
+        cachedPartitions.mustAdd(new CachedPartition(part.topicPartition, part.topicId, reqData, respData))
       }
-      (cachedPartitions, sessionTopicIds)
+      cachedPartitions
     }
     val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower,
         updates.size, usesTopicIds, () => createNewSession)
     if (hasInconsistentTopicIds) {
-      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP, Collections.emptyMap())
+      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP)

Review comment:
       ^ this is still something we need to resolve.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);

Review comment:
       ah good catch.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       We should have a test in the Fetcher which ensure that the builder received the correct information. Then we could have one for the request which ensure that the builder does its job correctly as well.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Right. It seems to be that the `canEqual(this)` does not make any sense here. Could you double check?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -413,8 +413,20 @@ abstract class AbstractFetcherThread(name: String,
 
                 case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
                   warn(s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
-                       "This error may be returned transiently when the partition is being created or deleted, but it is not " +
-                       "expected to persist.")
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_TOPIC_ID =>
+                  warn(s"Received ${Errors.UNKNOWN_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.INCONSISTENT_TOPIC_ID =>
+                  warn(s"Received ${Errors.INCONSISTENT_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")

Review comment:
       I don't think processFetchRequest is tested anywhere. There tests for the much higher level method doWork, so I can try to write one like that and check if there is that partition with error?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       I think I have the same confusion here as I do for the fetcher tests. I agree that changes should be tested, but I'm not really sure how to do this here.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       I guess the part I didn't understand is that buildFetch's builder is tested in FetchSessionHandler tests. But I guess there is one more method call we can test.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -301,7 +299,9 @@ public FetchRequestData build() {
                 if (nextData != null) {
                     // We basically check if the new partition had the same topic ID. If not,
                     // we add it to the "replaced" set.
-                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                    if (!prevData.topicId.equals(nextData.topicId)
+                            && !prevData.topicId.equals(Uuid.ZERO_UUID)
+                            && !nextData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       Is there a reason we do this? If the previous data had a topic ID and this one doesn't we should send a different fetch request version and the session will be closed.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       Right. Here I would like to have tests which ensure that the Builder is fed correctly based on the FetchSessionHandler's data.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -340,11 +340,9 @@ public FetchRequestData build() {
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
-                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
-
-                // There should only be one topic name per topic ID.
-                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+                sessionTopicNames = topicNames;
+            } else {
+                sessionTopicNames = Collections.emptyMap();

Review comment:
       I think the session will already have an empty map or close but it don't think it makes a big difference with or without this change.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));

Review comment:
       Unless you are referring to the sessionTopicNames line. 😅




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)

Review comment:
       Is this different than assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Hmm, how does my PR affect this?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -249,115 +215,126 @@ public String toString() {
          * Another reason is because we make use of the list ordering to optimize the preparation of
          * incremental fetch requests (see below).
          */
-        private LinkedHashMap<TopicPartition, PartitionData> next;
-        private Map<String, Uuid> topicIds;
+        private LinkedHashMap<TopicIdPartition, PartitionData> next;
         private final boolean copySessionPartitions;
         private int partitionsWithoutTopicIds = 0;
+        private int partitionsWithTopicIds = 0;
 
         Builder() {
             this.next = new LinkedHashMap<>();
-            this.topicIds = new HashMap<>();
             this.copySessionPartitions = true;
         }
 
         Builder(int initialSize, boolean copySessionPartitions) {
             this.next = new LinkedHashMap<>(initialSize);
-            this.topicIds = new HashMap<>(initialSize);
             this.copySessionPartitions = copySessionPartitions;
         }
 
         /**
          * Mark that we want data from this partition in the upcoming fetch.
          */
-        public void add(TopicPartition topicPartition, Uuid topicId, PartitionData data) {
-            next.put(topicPartition, data);
-            // topicIds should not change between adding partitions and building, so we can use putIfAbsent
-            if (!topicId.equals(Uuid.ZERO_UUID)) {
-                topicIds.putIfAbsent(topicPartition.topic(), topicId);
-            } else {
+        public void add(TopicIdPartition topicIdPartition, PartitionData data) {
+            next.put(topicIdPartition, data);
+            if (topicIdPartition.topicId().equals(Uuid.ZERO_UUID)) {
                 partitionsWithoutTopicIds++;
+            } else {
+                partitionsWithTopicIds++;
+            }
+        }
+
+        private Map<TopicIdPartition, PartitionData> buildFullSession(boolean canUseTopicIds) {
+            if (log.isDebugEnabled()) {
+                log.debug("Built full fetch {} for node {} with {}.",
+                        nextMetadata, node, partitionsToLogString(next.keySet()));
             }
+            sessionPartitions = next;
+            next = null;
+            // Only add topic IDs to the session if we are using topic IDs.
+            sessionTopicNames = new HashMap<>();
+            if (canUseTopicIds) {
+                Map<Uuid, Set<String>> newTopicNames = sessionPartitions.keySet().stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                sessionTopicNames = new HashMap<>(newTopicNames.size());
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+            } else {
+                sessionTopicNames = new HashMap<>();
+            }
+            return Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
         }
 
         public FetchRequestData build() {
             boolean canUseTopicIds = partitionsWithoutTopicIds == 0;
 
             if (nextMetadata.isFull()) {
-                if (log.isDebugEnabled()) {
-                    log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
-                }
-                sessionPartitions = next;
-                next = null;
-                // Only add topic IDs to the session if we are using topic IDs.
-                if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
-                } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
-                }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
-            for (Iterator<Entry<TopicPartition, PartitionData>> iter =
+            // If we were previously using a session without IDs and an ID was added to the builder, we will close the current session and open a new one with IDs.
+            // Same if vice versa.
+            boolean closeSessionDueToTopicIdChange = (requestUsedTopicIds && partitionsWithoutTopicIds > 0) || (!requestUsedTopicIds && partitionsWithTopicIds > 0);
+
+            if (closeSessionDueToTopicIdChange) {
+                canUseTopicIds = partitionsWithTopicIds > 0;
+                Map<TopicIdPartition, PartitionData> toSend = buildFullSession(canUseTopicIds);
+                if (canUseTopicIds && partitionsWithoutTopicIds == 0 || !canUseTopicIds && partitionsWithTopicIds == 0)
+                    return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata.nextCloseExisting(), !requestUsedTopicIds);
+                Map<TopicIdPartition, PartitionData> emptyMap = new LinkedHashMap<>();
+                return new FetchRequestData(emptyMap, Collections.emptyList(), emptyMap, nextMetadata.closeExisting(), !requestUsedTopicIds);
+            }
+
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            for (Iterator<Entry<TopicIdPartition, PartitionData>> iter =
                      sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
-                Entry<TopicPartition, PartitionData> entry = iter.next();
-                TopicPartition topicPartition = entry.getKey();
+                Entry<TopicIdPartition, PartitionData> entry = iter.next();
+                TopicIdPartition topicIdPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
-                PartitionData nextData = next.remove(topicPartition);
+                PartitionData nextData = next.remove(topicIdPartition);
                 if (nextData != null) {
                     if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
-                        next.put(topicPartition, nextData);
+                        next.put(topicIdPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(topicIdPartition);
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(topicIdPartition);

Review comment:
       I think we may even be able to get away with fewer maps. I see in the commit you have we add to topicIDs at the start but I'm not sure that works if we have more than one ID for a topic. I was thinking if we stored the ID in the fetch data, we wouldn't need to build a map from ids to names. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       In this case, it would be nice if we would have a `TopicIdPartition` which contains an optional topic name. For the context, the issue is that we might have partitions in the fetch requests for which the topic name is unknown or not yet known by the broker.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -378,53 +378,47 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
   * @param reqMetadata        The request metadata.
   * @param fetchData          The partition data from the fetch request.
   * @param usesTopicIds       True if this session should use topic IDs.
-  * @param topicIds           The map from topic names to topic IDs.
   * @param isFromFollower     True if this fetch request came from a follower.
   */
 class FullFetchContext(private val time: Time,
                        private val cache: FetchSessionCache,
                        private val reqMetadata: JFetchMetadata,
-                       private val fetchData: util.Map[TopicPartition, FetchRequest.PartitionData],
+                       private val fetchData: util.Map[TopicIdPartition, FetchRequest.PartitionData],
                        private val usesTopicIds: Boolean,
-                       private val topicIds: util.Map[String, Uuid],
                        private val isFromFollower: Boolean) extends FetchContext {
-  override def getFetchOffset(part: TopicPartition): Option[Long] =
+  override def getFetchOffset(part: TopicIdPartition): Option[Long] =
     Option(fetchData.get(part)).map(_.fetchOffset)
 
-  override def foreachPartition(fun: (TopicPartition, Uuid, FetchRequest.PartitionData) => Unit): Unit = {
-    fetchData.forEach((tp, data) => fun(tp, topicIds.get(tp.topic), data))
+  override def foreachPartition(fun: (TopicIdPartition, FetchRequest.PartitionData) => Unit): Unit = {
+    fetchData.forEach((tp, data) => fun(tp, data))
   }
 
   override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = {
-    FetchResponse.sizeOf(versionId, updates.entrySet.iterator, topicIds)
+    FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
   }
 
   override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
     var hasInconsistentTopicIds = false
-    def createNewSession: (FetchSession.CACHE_MAP, FetchSession.TOPIC_ID_MAP) = {
+    def createNewSession: FetchSession.CACHE_MAP = {
       val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
-      val sessionTopicIds = new util.HashMap[String, Uuid](updates.size)
       updates.forEach { (part, respData) =>
         if (respData.errorCode() == Errors.INCONSISTENT_TOPIC_ID.code()) {
           info(s"Session encountered an inconsistent topic ID for topicPartition $part.")
           hasInconsistentTopicIds = true
         }
         val reqData = fetchData.get(part)
-        val id = topicIds.getOrDefault(part.topic(), Uuid.ZERO_UUID)
-        cachedPartitions.mustAdd(new CachedPartition(part, id, reqData, respData))
-        if (id != Uuid.ZERO_UUID)
-          sessionTopicIds.put(part.topic, id)
+        cachedPartitions.mustAdd(new CachedPartition(part.topicPartition, part.topicId, reqData, respData))
       }
-      (cachedPartitions, sessionTopicIds)
+      cachedPartitions
     }
     val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower,
         updates.size, usesTopicIds, () => createNewSession)
     if (hasInconsistentTopicIds) {
-      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP, Collections.emptyMap())
+      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP)

Review comment:
       For the replica fetcher, we could choose not delay partitions with this error. Seems like in the fetcher, we just choose whether to update metadata. So maybe this won't be too difficult.
   
   Alternatively, we change the fetching flow to contain topic ID earlier in the process and so we can include in the error response as well. That would be a lot of work. 
   
   Still need to think through the current setup to make sure we aren't losing critical data in this state.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -233,7 +233,8 @@ class ReplicaFetcherThread(name: String,
         Map.empty
       }
     } else {
-      fetchResponse.responseData(fetchSessionHandler.sessionTopicNames, clientResponse.requestHeader().apiVersion()).asScala
+      fetchResponse.responseData(fetchSessionHandler.sessionTopicNames, clientResponse.requestHeader().apiVersion()).asScala.map {

Review comment:
       yes we will need to do 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +247,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)
+        // Update the topic name in place
+          cachedPart.resolveUnknownName(topicPart.topicPartition.topic)

Review comment:
       TODO: We can also put toForget back after the update step as we handle forgetting using different IDs.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Actually, you're right. That is not entirely true. I thought that the `requireNonNull` for the `topic` in one of the [constructor](https://github.com/apache/kafka/pull/11374/files#diff-3d6aa1dec2a2548f28148717926536cc937acec2ab4bd03a7bcdc58c84a6cbbaR38) would prevent this to work. However, as we use the other `TopicIdPartition` constructor here it is not impacted by the `requireNonNull`.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah. It's used in 49 places. Some of the places I intentionally left as zero uuids. I can convert all of them to Uuid.ZERO_UUID if we think this may be bug prone.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       The idea was to not do a put operation for every partition but instead every topic. Maybe grouping is slower though.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Good catch

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       FETCH_SESSION_TOPIC_ID_ERROR occurs when we switch from not using topic IDs in the request to using them (or vice versa). I think maybe we'd want to delay partitions to get the latest metadata, but not sure. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       Hmmm. So we'd sort out the ones with null names? What benefit are we thinking we'll get from this?
   

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       I think I wrote all of these before the class was updated. but i will change them. :)

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah. It's used in 49 places. Some of the places I intentionally left as zero uuids. I can convert all of them to Uuid.ZERO_UUID if we think this may be bug prone.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       This was here before my change, but I can remove it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       Hmmm. So we'd sort out the ones with null names? What benefit are we thinking we'll get from this?
   




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       FETCH_SESSION_TOPIC_ID_ERROR occurs when we switch from not using topic IDs in the request to using them (or vice versa). I think maybe we'd want to delay partitions to get the latest metadata, but not sure. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -202,29 +203,30 @@ public void testSessionless() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       I moved some back.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Hmm. I'm not quite sure why this would not make sense. I believe it is checking the types are correct.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)

Review comment:
       That could be 😄 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +935,341 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNamesFooChanged
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  private def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  private def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)
+            errorResponse(Errors.UNKNOWN_TOPIC_ID.code)
+          else
+            noErrorResponse
+        )
+      }
+      context.updateAndGenerateResponseData(data).sessionId
+    }
+
+    val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0))
+    val zar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("zar", 0))
+
+    val fooUnresolved = new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition))
+    val barUnresolved = new TopicIdPartition(bar.topicId, new TopicPartition(null, bar.partition))
+    val zarUnresolved = new TopicIdPartition(zar.topicId, new TopicPartition(null, zar.partition))
+
+    // The metadata cache does not know about the topic.
+    val context1 = newContext(
+      JFetchMetadata.INITIAL,
+      Seq(foo, bar, zar),
+      Map.empty[Uuid, String]
+    )
+
+    // So the context contains unresolved partitions.
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    assertPartitionsOrder(context1, Seq(fooUnresolved, barUnresolved, zarUnresolved))
+
+    // The response is sent back to create the session.
+    val sessionId = updateAndGenerateResponseData(context1)
+
+    // The metadata cache only knows about foo.
+    val context2 = newContext(
+      new JFetchMetadata(sessionId, 1),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic)
+    )
+
+    // So foo is resolved but not the others.
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    assertPartitionsOrder(context2, Seq(foo, barUnresolved, zarUnresolved))
+
+    updateAndGenerateResponseData(context2)
+
+    // The metadata cache knows about foo and bar.
+    val context3 = newContext(
+      new JFetchMetadata(sessionId, 2),
+      Seq(bar),
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic)
+    )
+
+    // So foo and bar are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context3.getClass)
+    assertPartitionsOrder(context3, Seq(foo, bar, zarUnresolved))
+
+    updateAndGenerateResponseData(context3)
+
+    // The metadata cache knows about all topics.
+    val context4 = newContext(
+      new JFetchMetadata(sessionId, 3),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic, zar.topicId -> zar.topic)
+    )
+
+    // So all topics are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context4.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+
+    updateAndGenerateResponseData(context4)
+
+    // The metadata cache does not know about the topics anymore (e.g. deleted).
+    val context5 = newContext(
+      new JFetchMetadata(sessionId, 4),
+      Seq.empty,
+      Map.empty
+    )
+
+    // All topics remain resolved.
+    assertEquals(classOf[IncrementalFetchContext], context5.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIdsInMetadataCache: Boolean, endsWithTopicIdsInMetadataCache: Boolean): Unit = {
+    // TODO: make cleaner
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIdsInMetadataCache) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNamesForRequest1
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val partitionsInSession1 = if (startsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context1, partitionsInSession1)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIdsInMetadataCache) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIdsInMetadataCache) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIdsInMetadataCache) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNamesForRequest2
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    // We will still have the topic ID in the session if we started with topic IDs and currently do not have the ID anymore.
+    val partitionsInSession2 = if (startsWithTopicIdsInMetadataCache || endsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context2, partitionsInSession2)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // If we always had topic IDs in the metadata cache or newly resolved the ID we won't have an error.
+    // If the topic ID was once in the metadata cache and in the session, the partition has likely been deleted and would have an INCONSISTENT_TOPIC_ID error.
+    // Likely if the topic ID was never in the broker, return UNKNOWN_TOPIC_OR_PARTITION
+    val errorCode2 = if (endsWithTopicIdsInMetadataCache) Errors.NONE.code
+    else if (startsWithTopicIdsInMetadataCache) Errors.INCONSISTENT_TOPIC_ID.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = partitionsInSession2(0)
+    val fooResponse2 = if (endsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)

Review comment:
       Is there a way to make such a test without duplicating the newContext portions? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       `buildFetch` seems to be well isolated so it should be quite easy to write a few unit tests for it. `buildFetch` returns a `Builder` so you will have to build the request in order to inspect it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +921,182 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNames
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // Likely if the topic ID is not in the broker, the partition has been deleted. In this case return UNKNOWN_TOPIC_OR_PARTITION
+    // If we started with unknown IDs and switched to having them, we won't have an error.
+    val errorCode2 = if (endsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = if (endsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse2 = if (endsWithTopicIds) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)
+
+    assertEquals(Errors.NONE, resp2.error)
     assertTrue(resp2.sessionId > 0)
     val responseData2 = resp2.responseData(topicNames, request2.version)
-    // We should have no partition responses with this top level error.
-    assertEquals(0, responseData2.size())
+    if (startsWithTopicIds && endsWithTopicIds) {
+      // if both requests had topic IDs there was no change so we won't have a response
+      assertEquals(0, responseData2.size())
+    } else {
+      assertEquals(errorCode2, responseData2.get(tp0).errorCode)
+    }
+  }
+
+  // This test simulates trying to forget a topic partition with all possible topic ID usages for both requests.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testToForgetCases(startsWithTopicIds: Boolean, endsWithTopicIds: Boolean): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIds) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNames
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIds) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIds) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIds) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request forgetting the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, Collections.singletonList(tidp0), false)
+    val topicNamesForRequest2 = if (endsWithTopicIds) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNames
+    )
+    // If we forget the last partition, we will have a sessionless context.
+    assertEquals(classOf[SessionlessFetchContext], context2.getClass)

Review comment:
       I think the issue with that approach is it doesn't quite cover the four cases, right? I could keep as is, but have a second partition that just uses IDs and resolve that one on the second round.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +235,31 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            addToForgottenTopicMap(removed, forgottenTopicMap);
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                addToForgottenTopicMap(replaced, forgottenTopicMap);
+            }

Review comment:
       Should we add a few unit tests to validate the changes that we have done in this class? We could add a few to FetchRequestTest (not use if it already exists though).




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -314,8 +356,7 @@ public int maxBytes() {
 
     // For versions < 13, builds the partitionData map using only the FetchRequestData.
     // For versions 13+, builds the partitionData map using both the FetchRequestData and a mapping of topic IDs to names.
-    // Throws UnknownTopicIdException for versions 13+ if the topic ID was unknown to the server.
-    public Map<TopicPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {
+    public Map<TopicIdPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {

Review comment:
       Do we have a unit test for this one and for `forgottenTopics`?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   System test failures are not related. Merged to trunk and to 3.1.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan edited a comment on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

Posted by GitBox <gi...@apache.org>.
jolshan edited a comment on pull request #11331:
URL: https://github.com/apache/kafka/pull/11331#issuecomment-949274575


   TODOs:
   1. ~Change inconsistent topic ID so it is no longer a top level error~
   2. ~Maybe refactor some of the receiving side code, we have a map with TopicIdPartition, PartitionData and both contain topic ID~ Decided to hold off on this as there are still usages for TopicIdPartition in the receiving side.
   3. Maybe change FetchSession to update newly unresolved partitions to no longer include topic name.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/DelayedFetch.scala
##########
@@ -92,7 +92,7 @@ class DelayedFetch(delayMs: Long,
         val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val partition = replicaManager.getPartitionOrException(topicPartition)
+            val partition = replicaManager.getPartitionOrException(topicPartition.topicPartition)

Review comment:
       Not sure if I should go through and rename some of these to `topicIdPartition`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -761,16 +836,16 @@ class FetchSessionTest {
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
 
     assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
       context2.updateAndGenerateResponseData(respData2).error())
   }
 
-  @Test
+  /*@Test

Review comment:
       I commented out this test, since I removed the behavior to catch inconsistent IDs at this stage. I can try to simulate catching the inconsistent ID later, but not sure if that is helpful to show in a test.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       It seems like right now `elementKeysAreEqual` is just `equals`. Is the idea in implementing this to prevent someone else from doing so and not using `equals`/the logic from equals?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +868,14 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
         response.data().responses().forEach { topicResponse =>
           topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+            val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), data.partitionIndex()))
+            // If the topic name was not known, we will have no bytes out.
+            if (tp.topicPartition.topic != null)

Review comment:
       Reassigning partitions takes a topic ID partition unfortunately. But I suppose we can change that. Not sure if we want to distinguish between reassigning partitions if we had two with the same name in the session.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +247,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)
+        // Update the topic name in place
+          cachedPart.resolveUnknownName(topicPart.topicPartition.topic)

Review comment:
       I'm not sure I follow here. We have an unresolved partition in the session and we are updating it. 
   Why would we not resolve the partition? I suppose it will get picked up by the forEach partition resolving process, but not sure how the earlier comment applies here.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       we should do that in addition to this method?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();
                 }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       Do we not care to change IDs if the data is equal?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +247,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)
+        // Update the topic name in place
+          cachedPart.resolveUnknownName(topicPart.topicPartition.topic)

Review comment:
       TODO 2: if we have an update with an unresolved name, should we change the name to be unresolved here? I think we should but want to confirm.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover all the methods that we have changed or added as well?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Let's create that file and put new unit tests there. That is the way it should be.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -340,11 +340,9 @@ public FetchRequestData build() {
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
-                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
-
-                // There should only be one topic name per topic ID.
-                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+                sessionTopicNames = topicNames;
+            } else {
+                sessionTopicNames = Collections.emptyMap();

Review comment:
       That is a good question. I thought that it is better to empty the map if we don't use topic ids instead of keeping a out-of-date mapping. What do you think?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       > append when the controller fails over to an older IBP during an upgrade.
   I think I'm misunderstanding something here. Did you mean to say append? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       I think that the grouping is slower because it has to allocate another Map, Sets for each Uuid, etc.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       So #11459 doesn't touch the FetchSessionHandler code. But I can still add these cases.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData, so I feel like that is sufficient unless I'm missing something.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       I can do that but it will take some time. 😅




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)

Review comment:
       Gotcha. I missed it. Changing the order to be consistent makes sense.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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


   System test results: http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2021-11-10--001.system-test-kafka-branch-builder--1636543025--jolshan--KAFKA-13111--165a106bf3/report.html
   
   A previous run was all green, so will need to confirm the 3 failed tests are unrelated to this change.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -270,36 +273,320 @@ public void testFetchNormal() {
     }
 
     @Test
-    public void testFetchWithNoId() {
+    public void testFetchWithNoTopicId() {
         // Should work and default to using old request type.
         buildFetcher();
 
-        TopicPartition noId = new TopicPartition("noId", 0);
-        TopicIdPartition noIdPart = new TopicIdPartition(Uuid.ZERO_UUID, noId);
-        assignFromUserNoId(singleton(noId));
-        subscriptions.seek(noId, 0);
+        TopicIdPartition noId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("noId", 0));
+        assignFromUserNoId(singleton(noId.topicPartition()));
+        subscriptions.seek(noId.topicPartition(), 0);
 
-        // fetch should use request version 12
+        // Fetch should use request version 12
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fullFetchResponse(noIdPart, this.records, Errors.NONE, 100L, 0));
+        client.prepareResponse(
+            fetchRequestMatcher((short) 12, noId, 0, Optional.of(validLeaderEpoch)),
+            fullFetchResponse(noId, this.records, Errors.NONE, 100L, 0)
+        );
         consumerClient.poll(time.timer(0));
         assertTrue(fetcher.hasCompletedFetches());
 
         Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
-        assertTrue(partitionRecords.containsKey(noId));
+        assertTrue(partitionRecords.containsKey(noId.topicPartition()));
 
-        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(noId);
+        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(noId.topicPartition());
         assertEquals(3, records.size());
-        assertEquals(4L, subscriptions.position(noId).offset); // this is the next fetching position
+        assertEquals(4L, subscriptions.position(noId.topicPartition()).offset); // this is the next fetching position
         long offset = 1;
         for (ConsumerRecord<byte[], byte[]> record : records) {
             assertEquals(offset, record.offset());
             offset += 1;
         }
     }
 
+    @Test
+    public void testFetchWithTopicId() {
+        buildFetcher();
+
+        TopicIdPartition tp = new TopicIdPartition(topicId, new TopicPartition(topicName, 0));
+        assignFromUser(singleton(tp.topicPartition()));
+        subscriptions.seek(tp.topicPartition(), 0);
+
+        // Fetch should use latest version
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(), tp, 0, Optional.of(validLeaderEpoch)),
+            fullFetchResponse(tp, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+
+        Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
+        assertTrue(partitionRecords.containsKey(tp.topicPartition()));
+
+        List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(tp.topicPartition());
+        assertEquals(3, records.size());
+        assertEquals(4L, subscriptions.position(tp.topicPartition()).offset); // this is the next fetching position
+        long offset = 1;
+        for (ConsumerRecord<byte[], byte[]> record : records) {
+            assertEquals(offset, record.offset());
+            offset += 1;
+        }
+    }
+
+    @Test
+    public void testFetchForgetTopicIdWhenUnassigned() {
+        buildFetcher();
+
+        TopicIdPartition foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+        TopicIdPartition bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0));
+
+        // Assign foo and bar.
+        subscriptions.assignFromUser(singleton(foo.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(foo), tp -> validLeaderEpoch));
+        subscriptions.seek(foo.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(foo, new PartitionData(
+                    foo.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, foo, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Assign bar and unassign foo.
+        subscriptions.assignFromUser(singleton(bar.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(bar), tp -> validLeaderEpoch));
+        subscriptions.seek(bar.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(bar, new PartitionData(
+                    bar.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                singletonList(foo)
+            ),
+            fullFetchResponse(1, bar, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+    }
+
+    @Test
+    public void testFetchForgetTopicIdWhenReplaced() {
+        buildFetcher();
+
+        TopicIdPartition fooWithOldTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+        TopicIdPartition fooWithNewTopicId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+
+        // Assign foo with old topic id.
+        subscriptions.assignFromUser(singleton(fooWithOldTopicId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithOldTopicId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithOldTopicId.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(fooWithOldTopicId, new PartitionData(
+                    fooWithOldTopicId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, fooWithOldTopicId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Replace foo with old topic id with foo with new topic id.
+        subscriptions.assignFromUser(singleton(fooWithNewTopicId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithNewTopicId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithNewTopicId.topicPartition(), 0);
+
+        // Fetch should use latest version.
+        assertEquals(1, fetcher.sendFetches());
+        assertFalse(fetcher.hasCompletedFetches());
+
+        // foo with old topic id should be removed from the session.
+        client.prepareResponse(
+            fetchRequestMatcher(ApiKeys.FETCH.latestVersion(),
+                singletonMap(fooWithNewTopicId, new PartitionData(
+                    fooWithNewTopicId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                singletonList(fooWithOldTopicId)
+            ),
+            fullFetchResponse(1, fooWithNewTopicId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+    }
+
+    @Test
+    public void testFetchTopicIdUpgradeDowngrade() {
+        buildFetcher();
+
+        TopicIdPartition fooWithoutId = new TopicIdPartition(Uuid.ZERO_UUID, new TopicPartition("foo", 0));
+        TopicIdPartition fooWithId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
+
+        // Assign foo without a topic id.
+        subscriptions.assignFromUser(singleton(fooWithoutId.topicPartition()));
+        client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, singleton(fooWithoutId), tp -> validLeaderEpoch));
+        subscriptions.seek(fooWithoutId.topicPartition(), 0);
+
+        // Fetch should use version 12.
+        assertEquals(1, fetcher.sendFetches());
+
+        client.prepareResponse(
+            fetchRequestMatcher((short) 12,
+                singletonMap(fooWithoutId, new PartitionData(
+                    fooWithoutId.topicId(),
+                    0,
+                    FetchRequest.INVALID_LOG_START_OFFSET,
+                    fetchSize,
+                    Optional.of(validLeaderEpoch))
+                ),
+                emptyList()
+            ),
+            fullFetchResponse(1, fooWithoutId, this.records, Errors.NONE, 100L, 0)
+        );
+        consumerClient.poll(time.timer(0));
+        assertTrue(fetcher.hasCompletedFetches());
+        fetchedRecords();
+
+        // Upgrade.
+        subscriptions.assignFromUser(singleton(fooWithId.topicPartition()));

Review comment:
       For my understanding, is this line necessary? We are assigning the same topic partition, right?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -2440,6 +2442,57 @@ class KafkaApisTest {
     assertNull(partitionData.abortedTransactions)
   }
 
+  /**
+   * Verifies that partitions with unknown topic ID errors are added to the erroneous set and there is not an attempt to fetch them.
+   */
+  @Test
+  def testFetchRequestErroneousPartitions(): Unit = {
+    val tidp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val tp = tidp.topicPartition
+    val nullTp = new TopicPartition(null, tp.partition)
+    addTopicToMetadataCache(tp.topic, 1, topicId = tidp.topicId)
+
+    expect(replicaManager.getLogConfig(EasyMock.eq(nullTp))).andReturn(None)
+
+    // Simulate unknown topic ID in the context
+    val fetchData = Map(new TopicIdPartition(tidp.topicId(), new TopicPartition(null, tidp.partition)) ->
+      new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000, Optional.empty())).asJava
+    val fetchDataBuilder = Map(tp -> new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000,
+      Optional.empty())).asJava
+    val fetchMetadata = new JFetchMetadata(0, 0)
+    val fetchContext = new FullFetchContext(time, new FetchSessionCache(1000, 100),
+      fetchMetadata, fetchData, false, false)
+    expect(fetchManager.newContext(
+      anyObject[Short],
+      anyObject[JFetchMetadata],
+      anyObject[Boolean],
+      anyObject[util.Map[TopicIdPartition, FetchRequest.PartitionData]],
+      anyObject[util.List[TopicIdPartition]],
+      anyObject[util.Map[Uuid, String]])).andReturn(fetchContext)

Review comment:
       We usually prefer to not use `any*` but to rather provide the expected values.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1732,197 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(tp1.topicId, 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(tp2.topicId, 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(tp3.topicId, 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  @Test
+  def testCachedPartitionEqualsAndHashCode(): Unit = {
+    val topicId = Uuid.randomUuid()
+    val topicName = "topic"
+    val partition = 0
+
+    val cachedPartitionWithIdAndName = new CachedPartition(topicName, topicId, partition)
+    val cachedPartitionWithIdAndNoName = new CachedPartition(null, topicId, partition)
+    val cachedPartitionWithDifferentIdAndName = new CachedPartition(topicName, Uuid.randomUuid(), partition)
+    val cachedPartitionWithZeroIdAndName = new CachedPartition(topicName, Uuid.ZERO_UUID, partition)
+    val cachedPartitionWithZeroIdAndOtherName = new CachedPartition("otherTopic", Uuid.ZERO_UUID, partition)
+
+    // CachedPartitions with valid topic IDs will compare topic ID and partition but not topic name.
+    assertTrue(cachedPartitionWithIdAndName.equals(cachedPartitionWithIdAndNoName))

Review comment:
       nit: You could use `assertEquals` as it calls `equals`.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +935,341 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNamesFooChanged
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  private def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  private def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)
+            errorResponse(Errors.UNKNOWN_TOPIC_ID.code)
+          else
+            noErrorResponse
+        )
+      }
+      context.updateAndGenerateResponseData(data).sessionId
+    }
+
+    val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0))
+    val zar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("zar", 0))
+
+    val fooUnresolved = new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition))
+    val barUnresolved = new TopicIdPartition(bar.topicId, new TopicPartition(null, bar.partition))
+    val zarUnresolved = new TopicIdPartition(zar.topicId, new TopicPartition(null, zar.partition))
+
+    // The metadata cache does not know about the topic.
+    val context1 = newContext(
+      JFetchMetadata.INITIAL,
+      Seq(foo, bar, zar),
+      Map.empty[Uuid, String]
+    )
+
+    // So the context contains unresolved partitions.
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    assertPartitionsOrder(context1, Seq(fooUnresolved, barUnresolved, zarUnresolved))
+
+    // The response is sent back to create the session.
+    val sessionId = updateAndGenerateResponseData(context1)
+
+    // The metadata cache only knows about foo.
+    val context2 = newContext(
+      new JFetchMetadata(sessionId, 1),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic)
+    )
+
+    // So foo is resolved but not the others.
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    assertPartitionsOrder(context2, Seq(foo, barUnresolved, zarUnresolved))
+
+    updateAndGenerateResponseData(context2)
+
+    // The metadata cache knows about foo and bar.
+    val context3 = newContext(
+      new JFetchMetadata(sessionId, 2),
+      Seq(bar),
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic)
+    )
+
+    // So foo and bar are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context3.getClass)
+    assertPartitionsOrder(context3, Seq(foo, bar, zarUnresolved))
+
+    updateAndGenerateResponseData(context3)
+
+    // The metadata cache knows about all topics.
+    val context4 = newContext(
+      new JFetchMetadata(sessionId, 3),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic, zar.topicId -> zar.topic)
+    )
+
+    // So all topics are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context4.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+
+    updateAndGenerateResponseData(context4)
+
+    // The metadata cache does not know about the topics anymore (e.g. deleted).
+    val context5 = newContext(
+      new JFetchMetadata(sessionId, 4),
+      Seq.empty,
+      Map.empty
+    )
+
+    // All topics remain resolved.
+    assertEquals(classOf[IncrementalFetchContext], context5.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIdsInMetadataCache: Boolean, endsWithTopicIdsInMetadataCache: Boolean): Unit = {
+    // TODO: make cleaner
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIdsInMetadataCache) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNamesForRequest1
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val partitionsInSession1 = if (startsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context1, partitionsInSession1)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIdsInMetadataCache) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIdsInMetadataCache) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIdsInMetadataCache) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNamesForRequest2
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    // We will still have the topic ID in the session if we started with topic IDs and currently do not have the ID anymore.
+    val partitionsInSession2 = if (startsWithTopicIdsInMetadataCache || endsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context2, partitionsInSession2)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // If we always had topic IDs in the metadata cache or newly resolved the ID we won't have an error.
+    // If the topic ID was once in the metadata cache and in the session, the partition has likely been deleted and would have an INCONSISTENT_TOPIC_ID error.
+    // Likely if the topic ID was never in the broker, return UNKNOWN_TOPIC_OR_PARTITION
+    val errorCode2 = if (endsWithTopicIdsInMetadataCache) Errors.NONE.code
+    else if (startsWithTopicIdsInMetadataCache) Errors.INCONSISTENT_TOPIC_ID.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = partitionsInSession2(0)
+    val fooResponse2 = if (endsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)

Review comment:
       I am not sure that I understand the value that we get out of this logic. `updateAndGenerateResponseData` creates a FetchResponse based on its input. Therefore, the response that we assert is not so surprising in the end, right? It will contain `INCONSISTENT_TOPIC_ID` if when the method gets it as an input. This logic would make sense for a test which verifies `updateAndGenerateResponseData` but looks like a distraction in a test which verify the name resolution logic. Am I missing something here?

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -2440,6 +2442,57 @@ class KafkaApisTest {
     assertNull(partitionData.abortedTransactions)
   }
 
+  /**
+   * Verifies that partitions with unknown topic ID errors are added to the erroneous set and there is not an attempt to fetch them.
+   */
+  @Test
+  def testFetchRequestErroneousPartitions(): Unit = {
+    val tidp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val tp = tidp.topicPartition
+    val nullTp = new TopicPartition(null, tp.partition)
+    addTopicToMetadataCache(tp.topic, 1, topicId = tidp.topicId)
+
+    expect(replicaManager.getLogConfig(EasyMock.eq(nullTp))).andReturn(None)
+
+    // Simulate unknown topic ID in the context
+    val fetchData = Map(new TopicIdPartition(tidp.topicId(), new TopicPartition(null, tidp.partition)) ->
+      new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000, Optional.empty())).asJava
+    val fetchDataBuilder = Map(tp -> new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000,
+      Optional.empty())).asJava
+    val fetchMetadata = new JFetchMetadata(0, 0)
+    val fetchContext = new FullFetchContext(time, new FetchSessionCache(1000, 100),
+      fetchMetadata, fetchData, false, false)
+    expect(fetchManager.newContext(
+      anyObject[Short],
+      anyObject[JFetchMetadata],
+      anyObject[Boolean],
+      anyObject[util.Map[TopicIdPartition, FetchRequest.PartitionData]],
+      anyObject[util.List[TopicIdPartition]],
+      anyObject[util.Map[Uuid, String]])).andReturn(fetchContext)
+
+    EasyMock.expect(clientQuotaManager.maybeRecordAndGetThrottleTimeMs(
+      anyObject[RequestChannel.Request](), anyDouble, anyLong)).andReturn(0)
+
+    val fetchRequest = new FetchRequest.Builder(ApiKeys.FETCH.latestVersion, ApiKeys.FETCH.latestVersion,
+      -1, 100, 0, fetchDataBuilder).build()
+    val request = buildRequest(fetchRequest)
+    val capturedResponse = expectNoThrottling(request)
+
+    EasyMock.replay(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, fetchManager)
+    createKafkaApis().handleFetchRequest(request)

Review comment:
       We have two paths (fetch from follower and fetch from consumer) in `handleFetchRequest` where we handle unknown topic names. Should we parameterize the test to cover both of them?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)

Review comment:
       Should this be a check when topic is null?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());

Review comment:
       assertEquals for the toSend/toReplace lists/map?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       @jolshan With Ismael's PR (https://github.com/apache/kafka/pull/11374), this trick does not work any more. We need to think about an alternative/better approach. 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       Or is it that the javadoc says things like `key.elementKeysAreEqual(e) and key.hashCode() == e.hashCode()` so we should be using elementKeysAreEqual in FetchSession?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1542,113 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(topicIds.get("foo"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(topicIds.get("bar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(topicIds.get("zar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicIdPartition]): Unit = {
+    val partitionsInContext = ArrayBuffer.empty[TopicIdPartition]
+    context.foreachPartition { (tp, _) =>
       partitionsInContext += tp
     }
     assertEquals(partitions, partitionsInContext.toSeq)
   }
 }
+
+object FetchSessionTest {
+  def idUsageCombinations: java.util.stream.Stream[Arguments] = {
+    Seq(
+      Arguments.of(true, true),
+      Arguments.of(true, false),
+      Arguments.of(false, true),
+      Arguments.of(false, false)
+    ).asJava.stream()
+  }
+}

Review comment:
       I can add some for the equals and hash methods in CachedPartition. What classes were you thinking of for others?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       What logic are we thinking? Checking that the unresolved topics are handled correctly?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       nit: Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       Do we have tests verifying this change?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       Could we iterate over `sessionPartitions` and directly populate `sessionTopicNames` by using `putIfAbsent` or even `put`? The grouping seems unnecessary to me here unless I am missing something.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));

Review comment:
       As `toSend` is not used before L288, how about putting this line over there?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
+                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
+
+                // There should only be one topic name per topic ID.
+                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
             }
 
             if (log.isDebugEnabled()) {
-                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {} " +
-                          "out of {}", nextMetadata, node, partitionsToLogString(added),
-                          partitionsToLogString(altered), partitionsToLogString(removed),
-                          partitionsToLogString(sessionPartitions.keySet()));
+                log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {}, " +
+                                "replaced {} out of {}", nextMetadata, node, topicIdPartitionsToLogString(added),

Review comment:
       nit: Could we align like it was before?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -346,38 +334,36 @@ public FetchRequestData build() {
                     break;
                 }
                 sessionPartitions.put(topicPartition, nextData);
-                added.add(topicPartition);
+                added.add(new TopicIdPartition(nextData.topicId, topicPartition));
             }
 
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                for (Map.Entry<String, Uuid> topic : topicIds.entrySet()) {
-                    String topicName = topic.getKey();
-                    Uuid addedId = topic.getValue();
-                    sessionTopicIds.put(topicName, addedId);
-                    sessionTopicNames.put(addedId, topicName);
-                }
+                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,

Review comment:
       Same comment as before.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchMetadata.java
##########
@@ -120,6 +120,13 @@ public FetchMetadata nextCloseExisting() {
         return new FetchMetadata(sessionId, INITIAL_EPOCH);
     }
 
+    /**
+     * Return the metadata for the next closed session response.
+     */
+    public FetchMetadata closeExisting() {

Review comment:
       It seems that this method is not used anymore. Could we remove it?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       `that.canEqual(this)` seems weird to me. It seems that we could just remove it.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)

Review comment:
       nit: We could add another constructor which takes a `TopicIdPartition`.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       nit: There is an extra space after `== null`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +864,15 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
-        response.data().responses().forEach { topicResponse =>
-          topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+        response.data.responses.forEach { topicResponse =>
+          topicResponse.partitions.forEach { data =>
+            // If the topic name was not known, we will have no bytes out.
+            if (topicResponse.topic != null) {
+              val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, data.partitionIndex()))

Review comment:
       nit: Parenthesis after `partitionIndex` could be omitted.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -3497,14 +3493,13 @@ object KafkaApis {
   // TODO: remove resolvedResponseData method when sizeOf can take a data object.
   private[server] def sizeOfThrottledPartitions(versionId: Short,
                                                 unconvertedResponse: FetchResponse,
-                                                quota: ReplicationQuotaManager,
-                                                topicIds: util.Map[String, Uuid]): Int = {
-    val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+                                                quota: ReplicationQuotaManager): Int = {
+    val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     unconvertedResponse.data.responses().forEach(topicResponse =>
       topicResponse.partitions().forEach(partition =>
-        responseData.put(new TopicPartition(topicResponse.topic(), partition.partitionIndex()), partition)))
+        responseData.put(new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), partition.partitionIndex())), partition)))

Review comment:
       nit: Parenthesis after partitionIndex could be omitted.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1186,7 +1174,7 @@ class ReplicaManager(val config: KafkaConfig,
             lastStableOffset = None,
             exception = Some(e))
         case e: Throwable =>
-          brokerTopicStats.topicStats(tp.topic).failedFetchRequestRate.mark()
+          brokerTopicStats.topicStats(tp.topicPartition.topic).failedFetchRequestRate.mark()

Review comment:
       nit: `tp.topic`

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
         }
       }
     } else {
       // Regular Kafka consumers need READ permission on each partition they are fetching.
-      val partitionDatas = new mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]
-      fetchContext.foreachPartition { (topicPartition, topicId, partitionData) =>
-        partitionDatas += topicPartition -> partitionData
-        sessionTopicIds.put(topicPartition.topic(), topicId)
-      }
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
-      partitionDatas.foreach { case (topicPartition, data) =>
-        if (!authorizedTopics.contains(topicPartition.topic))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
-        else if (!metadataCache.contains(topicPartition))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
+      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
+        if (topicIdPartition.topicPartition.topic == null)
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+        else
+          partitionDatas += topicIdPartition -> partitionData
+      }
+      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
+      partitionDatas.foreach { case (topicIdPartition, data) =>
+        if (!authorizedTopics.contains(topicIdPartition.topicPartition.topic))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        else if (!metadataCache.contains(topicIdPartition.topicPartition))
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
         else
-          interesting += (topicPartition -> data)
+          interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/DelayedFetch.scala
##########
@@ -92,7 +92,7 @@ class DelayedFetch(delayMs: Long,
         val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
         try {
           if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
-            val partition = replicaManager.getPartitionOrException(topicPartition)
+            val partition = replicaManager.getPartitionOrException(topicPartition.topicPartition)

Review comment:
       Yeah, that would be great. `topicPartition.topicPartition` looks really weird while reading.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode

Review comment:
       nit: Should we format the code as follow?
   
   ```
   override def hashCode: Int = {
     if (topicId != Uuid.ZERO_UUID)
       (31 * partition) + topicId.hashCode
     else
       (31 * partition) + topic.hashCode
   }
   ```

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Is this method still used? I can't find any usages of it.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -364,10 +405,7 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(forgottenTopic.topicId());
                         }
-                        if (name == null) {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", forgottenTopic.topicId()));
-                        }
-                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicPartition(name, partitionId)));
+                        forgottenTopic.partitions().forEach(partitionId -> toForget.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId))));

Review comment:
       I would also add a small comment here.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -328,31 +374,26 @@ public int maxBytes() {
                         } else {
                             name = topicNames.get(fetchTopic.topicId());
                         }
-                        if (name != null) {
-                            // If topic name is resolved, simply add to fetchData map
-                            fetchTopic.partitions().forEach(fetchPartition ->
-                                    fetchData.put(new TopicPartition(name, fetchPartition.partition()),
-                                            new PartitionData(
-                                                    fetchPartition.fetchOffset(),
-                                                    fetchPartition.logStartOffset(),
-                                                    fetchPartition.partitionMaxBytes(),
-                                                    optionalEpoch(fetchPartition.currentLeaderEpoch()),
-                                                    optionalEpoch(fetchPartition.lastFetchedEpoch())
-                                            )
-                                    )
-                            );
-                        } else {
-                            throw new UnknownTopicIdException(String.format("Topic Id %s in FetchRequest was unknown to the server", fetchTopic.topicId()));
-                        }
+                        fetchTopic.partitions().forEach(fetchPartition ->
+                                fetchData.put(new TopicIdPartition(fetchTopic.topicId(), new TopicPartition(name, fetchPartition.partition())),

Review comment:
       Should we add a comment here which explains that the topic name might be null in `TopicIdPartition` if we were unable to resolve it?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -432,9 +425,9 @@ private String partitionsToLogString(Collection<TopicPartition> partitions) {
     String verifyFullFetchResponsePartitions(Set<TopicPartition> topicPartitions, Set<Uuid> ids, short version) {
         StringBuilder bld = new StringBuilder();
         Set<TopicPartition> extra =
-            findMissing(topicPartitions, sessionPartitions.keySet());
+                findMissing(topicPartitions, sessionPartitions.keySet());

Review comment:
       nit: This change and the following ones do not seem necessary. I would revert them back.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Putting this here but it is not related to this line.
   
   It seems that we have an opportunity in `processFetchRequest` to better handle the `FETCH_SESSION_TOPIC_ID_ERROR` error. At the moment, it delays all the partitions. It seems to me that we could retry directly, no? If you agree, we could file a Jira and address this in a subsequent PR.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +222,49 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            removed.forEach(topicIdPartition -> {
+                FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                if (forgottenTopic == null) {
+                    forgottenTopic = new ForgottenTopic()
+                        .setTopic(topicIdPartition.topic())
+                        .setTopicId(topicIdPartition.topicId());
+                    forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                }
+                forgottenTopic.partitions().add(topicIdPartition.partition());
+            });
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                replaced.forEach(topicIdPartition -> {
+                    FetchRequestData.ForgottenTopic forgottenTopic = forgottenTopicMap.get(topicIdPartition.topic());
+                    if (forgottenTopic == null) {
+                        forgottenTopic = new ForgottenTopic()
+                            .setTopic(topicIdPartition.topic())
+                            .setTopicId(topicIdPartition.topicId());
+                        forgottenTopicMap.put(topicIdPartition.topic(), forgottenTopic);
+                    }
+                    forgottenTopic.partitions().add(topicIdPartition.partition());
+                });

Review comment:
       This block is identical to the previous one. Should we pull it into a helper method? (yeah, I know, I wrote this...)

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();

Review comment:
       Not related to this PR but could we use `Collections.emtpyMap` here? That would avoid allocating a `HashMap` all the times.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)

Review comment:
       nit: We can remove the parenthesis here.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I wonder if we should reply with `UNKNOWN_TOPIC_ID` for the topics whose are not resolved.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)

Review comment:
       nit: The if/else inline reads a bit weird. Should we extract the if/else?
   
   ```
   this.eq(that) || if (this.topicId != Uuid.ZERO_UUID)
     this.partition.equals(that.partition) && this.topicId.equals(that.topicId)
   else
     this.partition.equals(that.partition) && this.topic.equals(that.topic)
   ```

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)

Review comment:
       nit: It might be better to encapsulate this in `CachedPartition`. We could add a method called `maybeSetTopicName` or piggy back on `updateRequestParams`. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -801,23 +795,23 @@ class KafkaApis(val requestChannel: RequestChannel,
                 // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
                 // client.
                 new FetchResponseData.PartitionData()
-                  .setPartitionIndex(tp.partition)
+                  .setPartitionIndex(tp.topicPartition.partition)

Review comment:
       nit: We can use `tp.partition` here and a few other places.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)

Review comment:
       nit: How about naming it `cachedPartitionKey`? We could also benefits from passing `TopicIdPartition` to the constructor directly. 

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +263,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {

Review comment:
       Is `usesTopicIds` used anywhere in this method?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -224,10 +224,8 @@ class ReplicaFetcherThread(name: String,
     }
     val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
     if (!fetchSessionHandler.handleResponse(fetchResponse, clientResponse.requestHeader().apiVersion())) {
-      // If we had a topic ID related error, throw it, otherwise return an empty fetch data map.
-      if (fetchResponse.error == Errors.UNKNOWN_TOPIC_ID ||
-          fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR ||
-          fetchResponse.error == Errors.INCONSISTENT_TOPIC_ID) {
+      // If we had a session topic ID related error, throw it, otherwise return an empty fetch data map.
+      if (fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR) {

Review comment:
       I already mentioned this before but it seems that we could retry immediately in this case when the session was upgraded/downgraded. That would avoid having to wait for the backoff.

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       nit: `topicIdPartition.topic` should work.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Do we still use this constructor?

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1041,26 +1038,26 @@ class ReplicaManager(val config: KafkaConfig,
     //                        5) we found a diverging epoch
     if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData || hasDivergingEpoch) {
       val fetchPartitionData = logReadResults.map { case (tp, result) =>
-        val isReassignmentFetch = isFromFollower && isAddingReplica(tp, replicaId)
+        val isReassignmentFetch = isFromFollower && isAddingReplica(tp.topicPartition, replicaId)
         tp -> result.toFetchPartitionData(isReassignmentFetch)
       }
       responseCallback(fetchPartitionData)
     } else {
       // construct the fetch results from the read results
-      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicPartition, FetchPartitionStatus)]
-      fetchInfos.foreach { case (topicPartition, partitionData) =>
-        logReadResultMap.get(topicPartition).foreach(logReadResult => {
+      val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
+      fetchInfos.foreach { case (topicIdPartition, partitionData) =>
+        logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
           val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
-          fetchPartitionStatus += (topicPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
+          fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
       val fetchMetadata: SFetchMetadata = SFetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit,
-        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, topicIds, fetchPartitionStatus)
+        fetchOnlyFromLeader, fetchIsolation, isFromFollower, replicaId, fetchPartitionStatus)
       val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, clientMetadata,
         responseCallback)
 
       // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
+      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp.topicPartition) }

Review comment:
       nit: We could add another `apply` method to `TopicPartitionOperationKey` which accepts a `TopicIdPartition`. That will be convenient.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       I could theoretically check replace in the other test that checks multiple scenarios




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       It likely had something to do with how the mock client was handling metadata. But that may have been for the older version where we checked NodeApiVersion. I can try to switch it back.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       Looks like most of these changes were done by this commit: https://github.com/apache/kafka/pull/11331/commits/32c6297adb685f1863b8c7eb85f2f0965853a9f8
   so I can remove them pretty easily.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);
+
+        // Get the fetchData map from the request data.
+        Map<TopicIdPartition, FetchRequest.PartitionData> fetchData = fetchRequest.fetchData(topicNames);
+
+        // For fetch request version 13+ we expect topic names to be filled in for all topics in the topicNames map.
+        // Otherwise, the topic name should be null.
+        // For earlier request versions, we expect topic names and zero Uuids.
+        assertEquals(expectedFetchData.size(), fetchData.size());
+        expectedFetchData.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? topicNames.get(tidp.topicId()) : tidp.topic();
+            TopicIdPartition tpKey = new TopicIdPartition(tidp.topicId(), new TopicPartition(expectedName, tidp.partition()));
+            assertEquals(tidp.topicId(), fetchData.get(tpKey).topicId);
+        });

Review comment:
       I caught a bug with our PartitionData.equals method from implementing this. (We should be using .equals and not ==)




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/FetchRequestTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class FetchRequestTest {
+
+    private static Stream<Arguments> fetchVersions() {
+        return ApiKeys.FETCH.allVersions().stream().map(version -> Arguments.of(version));
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testToReplaceWithDifferentVersions(short version) {
+
+        Uuid topicId = Uuid.randomUuid();
+        TopicPartition tp = new TopicPartition("topic", 0);
+        TopicIdPartition tidp = new TopicIdPartition(topicId, tp);
+
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = Collections.singletonMap(tp,
+                new FetchRequest.PartitionData(topicId, 0, 0, 0, Optional.empty()));
+        List<TopicIdPartition> toReplace = Collections.singletonList(tidp);
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(toReplace)
+                .metadata(FetchMetadata.newIncremental(123)).build(version);
+
+        // If version < 13, we should not see any partitions in forgottenTopics. This is because we can not
+        // distinguish different topic IDs on versions earlier than 13.
+        assertEquals(fetchRequestUsesTopicIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @MethodSource("fetchVersions")
+    public void testFetchData(short version) {
+        TopicPartition topicPartition0 = new TopicPartition("topic", 0);
+        TopicPartition topicPartition1 = new TopicPartition("unknownIdTopic", 0);
+        Uuid topicId0 = Uuid.randomUuid();
+        Uuid topicId1 = Uuid.randomUuid();
+
+        // Only include topic IDs for the first topic partition.
+        Map<Uuid, String> topicNames = Collections.singletonMap(topicId0, topicPartition0.topic());
+        List<TopicIdPartition> topicIdPartitions = new LinkedList<>();
+        topicIdPartitions.add(new TopicIdPartition(topicId0, topicPartition0));
+        topicIdPartitions.add(new TopicIdPartition(topicId1, topicPartition1));
+
+        // Include one topic with topic IDs in the topic names map and one without.
+        Map<TopicPartition, FetchRequest.PartitionData> partitionData = new LinkedHashMap<>();
+        partitionData.put(topicPartition0, new FetchRequest.PartitionData(topicId0, 0, 0, 0, Optional.empty()));
+        partitionData.put(topicPartition1, new FetchRequest.PartitionData(topicId1, 0, 0, 0, Optional.empty()));
+        boolean fetchRequestUsesTopicIds = version >= 13;
+
+        FetchRequest fetchRequest = FetchRequest.parse(FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, partitionData)
+                .removed(Collections.emptyList())
+                .replaced(Collections.emptyList())
+                .metadata(FetchMetadata.newIncremental(123)).build(version).serialize(), version);
+
+        // For versions < 13, we will be provided a topic name and a zero UUID in FetchRequestData.
+        // Versions 13+ will contain a valid topic ID but an empty topic name.
+        List<TopicIdPartition> expectedFetchData = new LinkedList<>();
+        topicIdPartitions.forEach(tidp -> {
+            String expectedName = fetchRequestUsesTopicIds ? "" : tidp.topic();
+            Uuid expectedTopicId = fetchRequestUsesTopicIds ? tidp.topicId() : Uuid.ZERO_UUID;
+            expectedFetchData.add(new TopicIdPartition(expectedName, expectedTopicId, tidp.partition()));
+        });
+
+        // Build the list of TopicIdPartitions based on the FetchRequestData that was serialized and parsed.
+        List<TopicIdPartition> convertedFetchData = new LinkedList<>();
+        fetchRequest.data().topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                        convertedFetchData.add(new TopicIdPartition(topic.topic(), topic.topicId(), partition.partition()))
+                )
+        );
+        // The TopicIdPartitions built from the request data should match what we expect.
+        assertListEquals(expectedFetchData, convertedFetchData);

Review comment:
       I guess that it does not hurt to keep it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -2440,6 +2442,57 @@ class KafkaApisTest {
     assertNull(partitionData.abortedTransactions)
   }
 
+  /**
+   * Verifies that partitions with unknown topic ID errors are added to the erroneous set and there is not an attempt to fetch them.
+   */
+  @Test
+  def testFetchRequestErroneousPartitions(): Unit = {
+    val tidp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val tp = tidp.topicPartition
+    val nullTp = new TopicPartition(null, tp.partition)
+    addTopicToMetadataCache(tp.topic, 1, topicId = tidp.topicId)
+
+    expect(replicaManager.getLogConfig(EasyMock.eq(nullTp))).andReturn(None)
+
+    // Simulate unknown topic ID in the context
+    val fetchData = Map(new TopicIdPartition(tidp.topicId(), new TopicPartition(null, tidp.partition)) ->
+      new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000, Optional.empty())).asJava
+    val fetchDataBuilder = Map(tp -> new FetchRequest.PartitionData(tidp.topicId, 0, 0, 1000,
+      Optional.empty())).asJava
+    val fetchMetadata = new JFetchMetadata(0, 0)
+    val fetchContext = new FullFetchContext(time, new FetchSessionCache(1000, 100),
+      fetchMetadata, fetchData, false, false)
+    expect(fetchManager.newContext(
+      anyObject[Short],
+      anyObject[JFetchMetadata],
+      anyObject[Boolean],
+      anyObject[util.Map[TopicIdPartition, FetchRequest.PartitionData]],
+      anyObject[util.List[TopicIdPartition]],
+      anyObject[util.Map[Uuid, String]])).andReturn(fetchContext)

Review comment:
       I copied this from the test above. 😅  Wasn't sure if we wanted consistency amongst the tests.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -877,34 +935,341 @@ class FetchSessionTest {
 
     // Create an incremental fetch request as though no topics changed.
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     // Simulate ID changing on server.
     val topicNamesFooChanged =  Map(topicIds.get("bar") -> "bar", Uuid.randomUuid() -> "foo").asJava
-    val topicIdsFooChanged = topicNamesFooChanged.asScala.map(_.swap).asJava
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNamesFooChanged),
       request2.forgottenTopics(topicNamesFooChanged),
-      topicIdsFooChanged
+      topicNamesFooChanged
     )
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     // Likely if the topic ID is different in the broker, it will be different in the log. Simulate the log check finding an inconsistent ID.
-    respData2.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    respData2.put(tp0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
       .setHighWatermark(-1)
       .setLastStableOffset(-1)
       .setLogStartOffset(-1)
       .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
     val resp2 = context2.updateAndGenerateResponseData(respData2)
 
-    assertEquals(Errors.INCONSISTENT_TOPIC_ID, resp2.error)
+    assertEquals(Errors.NONE, resp2.error)
+    assertTrue(resp2.sessionId > 0)
+    val responseData2 = resp2.responseData(topicNames, request2.version)
+    // We should have the inconsistent topic ID error on the partition
+    assertEquals(Errors.INCONSISTENT_TOPIC_ID.code, responseData2.get(tp0.topicPartition).errorCode)
+  }
+
+  private def noErrorResponse: FetchResponseData.PartitionData = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setHighWatermark(10)
+      .setLastStableOffset(10)
+      .setLogStartOffset(10)
+  }
+
+  private def errorResponse(errorCode: Short): FetchResponseData.PartitionData  = {
+    new FetchResponseData.PartitionData()
+      .setPartitionIndex(0)
+      .setHighWatermark(-1)
+      .setLastStableOffset(-1)
+      .setLogStartOffset(-1)
+      .setErrorCode(errorCode)
+  }
+
+  @Test
+  def testResolveUnknownPartitions(): Unit = {
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+
+    def newContext(
+      metadata: JFetchMetadata,
+      partitions: Seq[TopicIdPartition],
+      topicNames: Map[Uuid, String] // Topic ID to name mapping known by the broker.
+    ): FetchContext = {
+      val data = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+      partitions.foreach { topicIdPartition =>
+        data.put(
+          topicIdPartition.topicPartition,
+          new FetchRequest.PartitionData(topicIdPartition.topicId, 0, 0, 100, Optional.empty())
+        )
+      }
+
+      val fetchRequest = createRequest(metadata, data, EMPTY_PART_LIST, false)
+
+      fetchManager.newContext(
+        fetchRequest.version,
+        fetchRequest.metadata,
+        fetchRequest.isFromFollower,
+        fetchRequest.fetchData(topicNames.asJava),
+        fetchRequest.forgottenTopics(topicNames.asJava),
+        topicNames.asJava
+      )
+    }
+
+    def updateAndGenerateResponseData(
+      context: FetchContext
+    ): Int = {
+      val data = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+      context.foreachPartition { (topicIdPartition, _) =>
+        data.put(
+          topicIdPartition,
+          if (topicIdPartition.topicId == Uuid.ZERO_UUID)
+            errorResponse(Errors.UNKNOWN_TOPIC_ID.code)
+          else
+            noErrorResponse
+        )
+      }
+      context.updateAndGenerateResponseData(data).sessionId
+    }
+
+    val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
+    val bar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0))
+    val zar = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("zar", 0))
+
+    val fooUnresolved = new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition))
+    val barUnresolved = new TopicIdPartition(bar.topicId, new TopicPartition(null, bar.partition))
+    val zarUnresolved = new TopicIdPartition(zar.topicId, new TopicPartition(null, zar.partition))
+
+    // The metadata cache does not know about the topic.
+    val context1 = newContext(
+      JFetchMetadata.INITIAL,
+      Seq(foo, bar, zar),
+      Map.empty[Uuid, String]
+    )
+
+    // So the context contains unresolved partitions.
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    assertPartitionsOrder(context1, Seq(fooUnresolved, barUnresolved, zarUnresolved))
+
+    // The response is sent back to create the session.
+    val sessionId = updateAndGenerateResponseData(context1)
+
+    // The metadata cache only knows about foo.
+    val context2 = newContext(
+      new JFetchMetadata(sessionId, 1),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic)
+    )
+
+    // So foo is resolved but not the others.
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    assertPartitionsOrder(context2, Seq(foo, barUnresolved, zarUnresolved))
+
+    updateAndGenerateResponseData(context2)
+
+    // The metadata cache knows about foo and bar.
+    val context3 = newContext(
+      new JFetchMetadata(sessionId, 2),
+      Seq(bar),
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic)
+    )
+
+    // So foo and bar are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context3.getClass)
+    assertPartitionsOrder(context3, Seq(foo, bar, zarUnresolved))
+
+    updateAndGenerateResponseData(context3)
+
+    // The metadata cache knows about all topics.
+    val context4 = newContext(
+      new JFetchMetadata(sessionId, 3),
+      Seq.empty,
+      Map(foo.topicId -> foo.topic, bar.topicId -> bar.topic, zar.topicId -> zar.topic)
+    )
+
+    // So all topics are resolved.
+    assertEquals(classOf[IncrementalFetchContext], context4.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+
+    updateAndGenerateResponseData(context4)
+
+    // The metadata cache does not know about the topics anymore (e.g. deleted).
+    val context5 = newContext(
+      new JFetchMetadata(sessionId, 4),
+      Seq.empty,
+      Map.empty
+    )
+
+    // All topics remain resolved.
+    assertEquals(classOf[IncrementalFetchContext], context5.getClass)
+    assertPartitionsOrder(context4, Seq(foo, bar, zar))
+  }
+
+  // This test either simulates an update to a partition using all possible topic ID usage combinations.
+  // The possible change will be found in an update from the partition.
+  @ParameterizedTest
+  @MethodSource(Array("idUsageCombinations"))
+  def testUpdatedPartitionResolvesId(startsWithTopicIdsInMetadataCache: Boolean, endsWithTopicIdsInMetadataCache: Boolean): Unit = {
+    // TODO: make cleaner
+    val time = new MockTime()
+    val cache = new FetchSessionCache(10, 1000)
+    val fetchManager = new FetchManager(time, cache)
+    val fooId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo").asJava
+    val tp0 = new TopicPartition("foo", 0)
+    val tidp0 = new TopicIdPartition(fooId, tp0)
+    val nullTidp0 = new TopicIdPartition(fooId, new TopicPartition(null, tp0.partition))
+
+    // Create a new fetch session with foo-0
+    val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData1.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    val topicNamesForRequest1 = if (startsWithTopicIdsInMetadataCache) topicNames else Map[Uuid, String]().asJava
+    // Start a fetch session. Simulate no error or unknown topic ID depending on whether we used topic IDs.
+    val context1 = fetchManager.newContext(
+      request1.version,
+      request1.metadata,
+      request1.isFromFollower,
+      request1.fetchData(topicNamesForRequest1),
+      request1.forgottenTopics(topicNamesForRequest1),
+      topicNamesForRequest1
+    )
+    assertEquals(classOf[FullFetchContext], context1.getClass)
+    val partitionsInSession1 = if (startsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context1, partitionsInSession1)
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    val errorCode1 = if (startsWithTopicIdsInMetadataCache) Errors.NONE.code else Errors.UNKNOWN_TOPIC_ID.code
+    val fooResponseTp1 = if (startsWithTopicIdsInMetadataCache) tidp0 else nullTidp0
+    val fooResponse1 = if (startsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode1)
+    respData1.put(fooResponseTp1, fooResponse1)
+    val resp1 = context1.updateAndGenerateResponseData(respData1)
+    assertEquals(Errors.NONE, resp1.error())
+    assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    val responseData1 = resp1.responseData(topicNames, request1.version)
+    assertEquals(errorCode1, responseData1.get(tp0).errorCode)
+
+    // Create an incremental fetch request with an update to the partition.
+    val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
+    reqData2.put(tp0, new FetchRequest.PartitionData(fooId, 0, 0, 100,
+      Optional.empty()))
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
+    val topicNamesForRequest2 = if (endsWithTopicIdsInMetadataCache) Map(fooId -> "foo").asJava else Map[Uuid, String]().asJava
+    val context2 = fetchManager.newContext(
+      request2.version,
+      request2.metadata,
+      request2.isFromFollower,
+      request2.fetchData(topicNamesForRequest2),
+      request2.forgottenTopics(topicNamesForRequest2),
+      topicNamesForRequest2
+    )
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)
+    // We will still have the topic ID in the session if we started with topic IDs and currently do not have the ID anymore.
+    val partitionsInSession2 = if (startsWithTopicIdsInMetadataCache || endsWithTopicIdsInMetadataCache) Seq(tidp0) else Seq(nullTidp0)
+    assertPartitionsOrder(context2, partitionsInSession2)
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    // If we always had topic IDs in the metadata cache or newly resolved the ID we won't have an error.
+    // If the topic ID was once in the metadata cache and in the session, the partition has likely been deleted and would have an INCONSISTENT_TOPIC_ID error.
+    // Likely if the topic ID was never in the broker, return UNKNOWN_TOPIC_OR_PARTITION
+    val errorCode2 = if (endsWithTopicIdsInMetadataCache) Errors.NONE.code
+    else if (startsWithTopicIdsInMetadataCache) Errors.INCONSISTENT_TOPIC_ID.code else Errors.UNKNOWN_TOPIC_OR_PARTITION.code
+    val fooResponseTp2 = partitionsInSession2(0)
+    val fooResponse2 = if (endsWithTopicIdsInMetadataCache) noErrorResponse else errorResponse(errorCode2)
+    respData2.put(fooResponseTp2, fooResponse2)
+    val resp2 = context2.updateAndGenerateResponseData(respData2)

Review comment:
       Do you mean the correct handling of the resolved partitions by `updateAndGenerateResponseData`? I think testing `updateAndGenerateResponseData` is a good thing in general. Perhaps, we should just put this into a separate test specific to that method? We should also test it for all context types with and without topic id, I guess.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       I suppose so. I wonder if we should even include the update method at all then...




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       What do you mean?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       The part I don't understand is that this building is in a method that sends the requests. I'm not sure how to pull that out and test specifically that the fetcher is getting the correct info. The fetcher is simply pulling from the FetchSessionHandler's build FetchRequestData.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -301,7 +299,9 @@ public FetchRequestData build() {
                 if (nextData != null) {
                     // We basically check if the new partition had the same topic ID. If not,
                     // we add it to the "replaced" set.
-                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                    if (!prevData.topicId.equals(nextData.topicId)
+                            && !prevData.topicId.equals(Uuid.ZERO_UUID)
+                            && !nextData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       I realize we may still want this as if the partition data is exactly the same, we will actually ignore the downgrade which is not good.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -301,7 +299,9 @@ public FetchRequestData build() {
                 if (nextData != null) {
                     // We basically check if the new partition had the same topic ID. If not,
                     // we add it to the "replaced" set.
-                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                    if (!prevData.topicId.equals(nextData.topicId)
+                            && !prevData.topicId.equals(Uuid.ZERO_UUID)
+                            && !nextData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       I have a test where this happens. 😅




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Ah I'm already doing this. 😅  Ok. sounds good.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -301,7 +299,9 @@ public FetchRequestData build() {
                 if (nextData != null) {
                     // We basically check if the new partition had the same topic ID. If not,
                     // we add it to the "replaced" set.
-                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                    if (!prevData.topicId.equals(nextData.topicId)
+                            && !prevData.topicId.equals(Uuid.ZERO_UUID)
+                            && !nextData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       Without this, when a topic id is set back to "zero", the former topic id is added to the replaced set which is a bit unintuitive, I think. In the end, it does not matter too much because the version is downgraded in this case so the replaced set is ignored. I was debating if it worth handling this case explicitly here.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       Sounds good, thanks!




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,125 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val fooId = Uuid.randomUuid()
+    val barId = Uuid.randomUuid()
+    val zarId = Uuid.randomUuid()
+    val topicNames = Map(fooId -> "foo", barId -> "bar").asJava
+    val foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1))
+    val zar0 = new TopicIdPartition(zarId, new TopicPartition("zar", 0))
+    val emptyFoo0 = new TopicIdPartition(fooId, new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(fooId, new TopicPartition(null, 1))
+    val emptyZar0 = new TopicIdPartition(zarId, new TopicPartition(null, 0))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    reqData1.put(zar0.topicPartition, new FetchRequest.PartitionData(zar0.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(barId, "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    assertPartitionsOrder(context1, Seq(emptyFoo0, emptyFoo1, emptyZar0))
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyZar0, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // Topic names in the session but not in the request are lazily resolved via foreachPartition. Resolve foo topic IDs here.
+    assertPartitionsOrder(context2, Seq(foo0, foo1, emptyZar0))
+    assertEquals(classOf[IncrementalFetchContext], context2.getClass)

Review comment:
       I can change the ordering of these asserts so they are consistent with the earlier ones




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -672,29 +672,22 @@ class KafkaApis(val requestChannel: RequestChannel,
     val versionId = request.header.apiVersion
     val clientId = request.header.clientId
     val fetchRequest = request.body[FetchRequest]
-    val (topicIds, topicNames) =
+    val topicNames =
       if (fetchRequest.version() >= 13)
-        metadataCache.topicIdInfo()
+        metadataCache.topicIdsToNames()
       else
-        (Collections.emptyMap[String, Uuid](), Collections.emptyMap[Uuid, String]())
+        Collections.emptyMap[Uuid, String]()
 
-    // If fetchData or forgottenTopics contain an unknown topic ID, return a top level error.
-    var fetchData: util.Map[TopicPartition, FetchRequest.PartitionData] = null
-    var forgottenTopics: util.List[TopicPartition] = null
-    try {
-      fetchData = fetchRequest.fetchData(topicNames)
-      forgottenTopics = fetchRequest.forgottenTopics(topicNames)
-    } catch {
-      case e: UnknownTopicIdException => throw e
-    }
+    val fetchData = fetchRequest.fetchData(topicNames)
+    val forgottenTopics = fetchRequest.forgottenTopics(topicNames)

Review comment:
       When a session is used, resolving the topic ids is not really necessary here because we should already have the names in the session or we would resolve them later anyway. I wonder if it would be better to do this entirely in the `fetchManager.newConext` based on the context type. Have you considered something like this?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,45 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     val sessionTopicIds = mutable.Map[String, Uuid]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          sessionTopicIds.put(topicIdPartition.topicPartition.topic, topicIdPartition.topicId)

Review comment:
       Do we still need this `sessionTopicIds` mapping if we have the topic id in the `topicIdPartition`?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,45 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     val sessionTopicIds = mutable.Map[String, Uuid]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          sessionTopicIds.put(topicIdPartition.topicPartition.topic, topicIdPartition.topicId)
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition.topicPartition.partition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition.topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          sessionTopicIds.put(topicIdPartition.topicPartition.topic, topicIdPartition.topicId)
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition.topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
         }
       }
     } else {
       // Regular Kafka consumers need READ permission on each partition they are fetching.
-      val partitionDatas = new mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]
-      fetchContext.foreachPartition { (topicPartition, topicId, partitionData) =>
-        partitionDatas += topicPartition -> partitionData
-        sessionTopicIds.put(topicPartition.topic(), topicId)
-      }
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
-      partitionDatas.foreach { case (topicPartition, data) =>
-        if (!authorizedTopics.contains(topicPartition.topic))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
-        else if (!metadataCache.contains(topicPartition))
-          erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
+      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
+        partitionDatas += topicIdPartition -> partitionData
+        sessionTopicIds.put(topicIdPartition.topicPartition.topic(), topicIdPartition.topicId)

Review comment:
       Could we direclty check if the topic name is null here and put the unresolved ones to `erroneous`? This would avoid the filter on the next line.

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -233,7 +233,8 @@ class ReplicaFetcherThread(name: String,
         Map.empty
       }
     } else {
-      fetchResponse.responseData(fetchSessionHandler.sessionTopicNames, clientResponse.requestHeader().apiVersion()).asScala
+      fetchResponse.responseData(fetchSessionHandler.sessionTopicNames, clientResponse.requestHeader().apiVersion()).asScala.map {

Review comment:
       Not related to this line. Don't wee need to update the fetcher to handle the topic id errors at the partition level?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,7 +173,8 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else

Review comment:
       Could we add a scaladoc for this method which explains what we do and why?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -378,53 +378,47 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
   * @param reqMetadata        The request metadata.
   * @param fetchData          The partition data from the fetch request.
   * @param usesTopicIds       True if this session should use topic IDs.
-  * @param topicIds           The map from topic names to topic IDs.
   * @param isFromFollower     True if this fetch request came from a follower.
   */
 class FullFetchContext(private val time: Time,
                        private val cache: FetchSessionCache,
                        private val reqMetadata: JFetchMetadata,
-                       private val fetchData: util.Map[TopicPartition, FetchRequest.PartitionData],
+                       private val fetchData: util.Map[TopicIdPartition, FetchRequest.PartitionData],
                        private val usesTopicIds: Boolean,
-                       private val topicIds: util.Map[String, Uuid],
                        private val isFromFollower: Boolean) extends FetchContext {
-  override def getFetchOffset(part: TopicPartition): Option[Long] =
+  override def getFetchOffset(part: TopicIdPartition): Option[Long] =
     Option(fetchData.get(part)).map(_.fetchOffset)
 
-  override def foreachPartition(fun: (TopicPartition, Uuid, FetchRequest.PartitionData) => Unit): Unit = {
-    fetchData.forEach((tp, data) => fun(tp, topicIds.get(tp.topic), data))
+  override def foreachPartition(fun: (TopicIdPartition, FetchRequest.PartitionData) => Unit): Unit = {
+    fetchData.forEach((tp, data) => fun(tp, data))
   }
 
   override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = {
-    FetchResponse.sizeOf(versionId, updates.entrySet.iterator, topicIds)
+    FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
   }
 
   override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
     var hasInconsistentTopicIds = false
-    def createNewSession: (FetchSession.CACHE_MAP, FetchSession.TOPIC_ID_MAP) = {
+    def createNewSession: FetchSession.CACHE_MAP = {
       val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
-      val sessionTopicIds = new util.HashMap[String, Uuid](updates.size)
       updates.forEach { (part, respData) =>
         if (respData.errorCode() == Errors.INCONSISTENT_TOPIC_ID.code()) {
           info(s"Session encountered an inconsistent topic ID for topicPartition $part.")
           hasInconsistentTopicIds = true
         }
         val reqData = fetchData.get(part)
-        val id = topicIds.getOrDefault(part.topic(), Uuid.ZERO_UUID)
-        cachedPartitions.mustAdd(new CachedPartition(part, id, reqData, respData))
-        if (id != Uuid.ZERO_UUID)
-          sessionTopicIds.put(part.topic, id)
+        cachedPartitions.mustAdd(new CachedPartition(part.topicPartition, part.topicId, reqData, respData))
       }
-      (cachedPartitions, sessionTopicIds)
+      cachedPartitions
     }
     val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower,
         updates.size, usesTopicIds, () => createNewSession)
     if (hasInconsistentTopicIds) {
-      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP, Collections.emptyMap())
+      FetchResponse.of(Errors.INCONSISTENT_TOPIC_ID, 0, responseSessionId, new FetchSession.RESP_MAP)

Review comment:
       Do we still need to return `INCONSISTENT_TOPIC_ID` a top level error? Fetcher prior to this change would need it, for sure. With this PR, we actually don't want the fetcher to treat it as a top level error but rather as a partition error. We need to think/discuss this a little more, I think.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -238,47 +247,40 @@ class FetchSession(val id: Int,
 
   def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) }
 
-  def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized {
-    Option(partitionMap.find(new CachedPartition(topicPartition,
-      sessionTopicIds.getOrDefault(topicPartition.topic(), Uuid.ZERO_UUID)))).map(_.fetchOffset)
+  def getFetchOffset(topicIdPartition: TopicIdPartition): Option[Long] = synchronized {
+    Option(partitionMap.find(new CachedPartition(topicIdPartition.topicPartition, topicIdPartition.topicId))).map(_.fetchOffset)
   }
 
-  type TL = util.ArrayList[TopicPartition]
+  type TL = util.ArrayList[TopicIdPartition]
 
   // Update the cached partition data based on the request.
   def update(fetchData: FetchSession.REQ_MAP,
-             toForget: util.List[TopicPartition],
+             toForget: util.List[TopicIdPartition],
              reqMetadata: JFetchMetadata,
-             topicIds: util.Map[String, Uuid]): (TL, TL, TL, TL) = synchronized {
+             usesTopicIds: Boolean): (TL, TL, TL) = synchronized {
     val added = new TL
     val updated = new TL
     val removed = new TL
-    val inconsistentTopicIds = new TL
     fetchData.forEach { (topicPart, reqData) =>
-      // Get the topic ID on the broker, if it is valid and the topic is new to the session, add its ID.
-      // If the topic already existed, check that its ID is consistent.
-      val id = topicIds.getOrDefault(topicPart.topic, Uuid.ZERO_UUID)
-      val newCachedPart = new CachedPartition(topicPart, id, reqData)
-      if (id != Uuid.ZERO_UUID) {
-        val prevSessionTopicId = sessionTopicIds.putIfAbsent(topicPart.topic, id)
-        if (prevSessionTopicId != null && prevSessionTopicId != id)
-          inconsistentTopicIds.add(topicPart)
-      }
+      val newCachedPart = new CachedPartition(topicPart.topicPartition, topicPart.topicId, reqData)
       val cachedPart = partitionMap.find(newCachedPart)
       if (cachedPart == null) {
         partitionMap.mustAdd(newCachedPart)
         added.add(topicPart)
       } else {
         cachedPart.updateRequestParams(reqData)
+        if (cachedPart.topic == null)
+        // Update the topic name in place
+          cachedPart.resolveUnknownName(topicPart.topicPartition.topic)

Review comment:
       This might not be necessary if we won't resolve topic ids in the request in all cases (see my previous comment).

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,45 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     val sessionTopicIds = mutable.Map[String, Uuid]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          sessionTopicIds.put(topicIdPartition.topicPartition.topic, topicIdPartition.topicId)
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition.topicPartition.partition, Errors.UNKNOWN_TOPIC_ID)

Review comment:
       nit: Could we add an overload to `partitionResponse` which takes a `TopicIdPartition`? This would reduce the boiler plate code a bit here. 

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -173,8 +184,8 @@ class CachedPartition(val topic: String,
         this.eq(that) ||
           (that.canEqual(this) &&
             this.partition.equals(that.partition) &&
-            this.topic.equals(that.topic) &&
-            this.topicId.equals(that.topicId))
+            (if (this.topicId != Uuid.ZERO_UUID) this.topicId.equals(that.topicId)
+            else this.topic.equals(that.topic)))

Review comment:
       Side note here: I think that we should implement `override def elementKeysAreEqual(that: Any): Boolean` from the `ImplicitLinkedHashCollection.Element` interface to make it clear that we do this for comparing elements in the collections.

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -870,12 +868,14 @@ class KafkaApis(val requestChannel: RequestChannel,
 
         // Prepare fetch response from converted data
         val response =
-          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, sessionTopicIds.asJava)
+          FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
         // record the bytes out metrics only when the response is being sent
         response.data().responses().forEach { topicResponse =>
           topicResponse.partitions().forEach { data =>
-            val tp = new TopicPartition(topicResponse.topic(), data.partitionIndex())
-            brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
+            val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic(), data.partitionIndex()))
+            // If the topic name was not known, we will have no bytes out.
+            if (tp.topicPartition.topic != null)

Review comment:
       Should we create `tp` after this check? We could also create a `TopicPartition` as we don't really use `TopicIdPartition` for the metric.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )

Review comment:
       We should probably remove that non null check, since it's weird to have it only in that one path. I can submit a 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala
##########
@@ -20,13 +20,13 @@ package kafka.server
 import kafka.log.LogConfig
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
-import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
-

Review comment:
       nit: spacing




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,45 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     val sessionTopicIds = mutable.Map[String, Uuid]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          sessionTopicIds.put(topicIdPartition.topicPartition.topic, topicIdPartition.topicId)

Review comment:
       Hmmm maybe not. Looks like I just put into this map but never get anything.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -413,8 +413,20 @@ abstract class AbstractFetcherThread(name: String,
 
                 case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
                   warn(s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
-                       "This error may be returned transiently when the partition is being created or deleted, but it is not " +
-                       "expected to persist.")
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_TOPIC_ID =>
+                  warn(s"Received ${Errors.UNKNOWN_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.INCONSISTENT_TOPIC_ID =>
+                  warn(s"Received ${Errors.INCONSISTENT_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")

Review comment:
       Yeah, that should work. Otherwise, we could also make the method package private and add a few unit tests for it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       I ended up making the new test file. I was confused because I thought the data object needed to be tested but it doesn't. I think this can be resolved.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       If we always resolve when iterating through the partitions, then do we need to resolve via the update method?




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       Nice. This works well.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));

Review comment:
       Yes, I was referring to `sessionTopicNames`.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +432,182 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    private static Stream<Arguments> idUsageCombinations() {
+        return Stream.of(
+                Arguments.of(true, true),
+                Arguments.of(true, false),
+                Arguments.of(false, true),
+                Arguments.of(false, false)
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("idUsageCombinations")
+    public void testTopicIdReplaced(boolean startsWithTopicIds, boolean endsWithTopicIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        builder.add(tp, new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123, respMap(new RespEntry("foo", 0, topicId1, 10, 20)));
+        short version = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+        handler.handleResponse(resp, version);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = endsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        if (startsWithTopicIds && endsWithTopicIds) {
+            // If we started with an ID, both a only a new ID will count towards replaced.
+            // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+            assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+            assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                    data2.toSend(), data2.sessionPartitions());
+            assertTrue(handler.sessionTopicNames().containsKey(topicId2));
+        } else if (startsWithTopicIds || endsWithTopicIds) {
+            // If we downgraded to not using topic IDs we will want to send this data.
+            // However, we will not mark the partition as one replaced. In this scenario, we should see the session close due to
+            // changing request types.
+            // We will have the new topic ID in the session partition map
+            assertEquals(0, data2.toReplace().size());
+            assertEquals(1, data2.toSend().size());

Review comment:
       Right.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -340,11 +340,9 @@ public FetchRequestData build() {
             // Add topic IDs to session if we can use them. If an ID is inconsistent, we will handle in the receiving broker.
             // If we switched from using topic IDs to not using them (or vice versa), that error will also be handled in the receiving broker.
             if (canUseTopicIds) {
-                Map<Uuid, Set<String>> newTopicNames = added.stream().collect(Collectors.groupingByConcurrent(TopicIdPartition::topicId,
-                        Collectors.mapping(topicIdPartition -> topicIdPartition.topicPartition().topic(), Collectors.toSet())));
-
-                // There should only be one topic name per topic ID.
-                newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
+                sessionTopicNames = topicNames;
+            } else {
+                sessionTopicNames = Collections.emptyMap();

Review comment:
       Do we need to reassign to empty map here? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       Alternatively I can just rewrite this.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       Yeah, that is not really necessary as you said. I don't mind if you remote it.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -967,6 +967,113 @@ class FetchSessionTest {
       .setErrorCode(errorCode)
   }
 
+  @Test

Review comment:
       Yeah, that is not really necessary as you said. I don't mind if you remove it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       I think I wrote all of these before the class was updated. but i will change them. :)




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Good catch




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2659,6 +2661,9 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
                 autoCommitIntervalMs,
                 interceptors,
                 throwOnStableOffsetNotSupported);
+        ApiVersions apiVersions = new ApiVersions();
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Nope. Looks like another change I forgot to cleanup.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Not quite sure what you meant here but I added this for now:
   `context1.foreachPartition((topicIdPartition, _) => assertEquals(topicIds.get("foo"), topicIdPartition.topicId))`




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       The idea was to not do a put operation for every partition but instead every topic. Maybe grouping is slower though.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Good catch

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       FETCH_SESSION_TOPIC_ID_ERROR occurs when we switch from not using topic IDs in the request to using them (or vice versa). I think maybe we'd want to delay partitions to get the latest metadata, but not sure. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       Hmmm. So we'd sort out the ones with null names? What benefit are we thinking we'll get from this?
   

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       I think I wrote all of these before the class was updated. but i will change them. :)

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah. It's used in 49 places. Some of the places I intentionally left as zero uuids. I can convert all of them to Uuid.ZERO_UUID if we think this may be bug prone.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       This was here before my change, but I can remove it.

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       The idea was to not do a put operation for every partition but instead every topic. Maybe grouping is slower though.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -824,6 +823,14 @@ static boolean hasUsableOffsetForLeaderEpochVersion(NodeApiVersions nodeApiVersi
         return OffsetsForLeaderEpochRequest.supportsTopicPermission(apiVersion.maxVersion());
     }
 
+    static boolean hasUsableTopicIdFetchRequestVersion(NodeApiVersions nodeApiVersions) {

Review comment:
       Good catch

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       FETCH_SESSION_TOPIC_ID_ERROR occurs when we switch from not using topic IDs in the request to using them (or vice versa). I think maybe we'd want to delay partitions to get the latest metadata, but not sure. 

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       Hmmm. So we'd sort out the ones with null names? What benefit are we thinking we'll get from this?
   

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -1021,17 +1018,17 @@ class ReplicaManager(val config: KafkaConfig,
     var bytesReadable: Long = 0
     var errorReadingData = false
     var hasDivergingEpoch = false
-    val logReadResultMap = new mutable.HashMap[TopicPartition, LogReadResult]
-    logReadResults.foreach { case (topicPartition, logReadResult) =>
-      brokerTopicStats.topicStats(topicPartition.topic).totalFetchRequestRate.mark()
+    val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
+    logReadResults.foreach { case (topicIdPartition, logReadResult) =>
+      brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()

Review comment:
       I think I wrote all of these before the class was updated. but i will change them. :)

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah. It's used in 49 places. Some of the places I intentionally left as zero uuids. I can convert all of them to Uuid.ZERO_UUID if we think this may be bug prone.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       This was here before my change, but I can remove it.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Yes, I was referring to those. Ack, I missed them during my first read.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       That is right.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       You could use `assertPartitionsOrder` helper here as well.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Should we add or extend a test in `FetcherTest` to cover this change? I would like to have one which ensure that the request sent is populated correctly (especially the replaced part) by the fetcher based on the session handler. It seems that we don't have such test in the suite at the moment.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Anyway, we don't need to address this in this PR. I just wanted to point out that there is an opportunity for an improvement.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       So you are asking for a test that is checking the fetcher builds the request correctly? Is this a test for the fetcher or the builder? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Yeah, I agree with you. Perhaps, we could just remove the maybeSetTopicName and move its logic into the update request params method.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       It is curious that we don't assert the forgotten partitions here. Is there a reason?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -212,6 +217,9 @@ private void assignFromUser(Set<TopicPartition> partitions) {
         metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWithIds("dummy", 1,
             Collections.emptyMap(), singletonMap(topicName, 4),
             tp -> validLeaderEpoch, topicIds), false, 0L);
+
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Do we still need this change?

##########
File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
##########
@@ -794,22 +793,23 @@ class ReplicaManagerTest {
 
       // We receive one valid request from the follower and replica state is updated
       var successfulFetch: Option[FetchPartitionData] = None
-      def callback(response: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
-        successfulFetch = response.headOption.filter { case (topicPartition, _) => topicPartition == tp }.map { case (_, data) => data }
+      def callback(response: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
+        // Check the topic partition only since we are reusing this callback on different TopicIdPartitions.
+        successfulFetch = response.headOption.filter { case (topicIdPartition, _) => topicIdPartition.topicPartition == tidp.topicPartition }.map { case (_, data) => data }

Review comment:
       This is not ideal. Could we validate that the topic id is correct as well?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2659,6 +2661,9 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
                 autoCommitIntervalMs,
                 interceptors,
                 throwOnStableOffsetNotSupported);
+        ApiVersions apiVersions = new ApiVersions();
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       Do we still need this change?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);

Review comment:
       Why do we use 12 here?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -202,29 +203,30 @@ public void testSessionless() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       nit: Is it worth bringing back this line on the previous one as there is space now? It might be too long though.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Would it be more appropriate to move the above assertions to `FetchRequestTest`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Should we also test when the current topic-partition in the session does not have a topic id? In this case, it should not be added to the `toReplace` set.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2069,10 +2071,10 @@ public void testReturnRecordsDuringRebalance() throws InterruptedException {
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
         ConsumerPartitionAssignor assignor = new CooperativeStickyAssignor();
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
-
         initMetadata(client, Utils.mkMap(Utils.mkEntry(topic, 1), Utils.mkEntry(topic2, 1), Utils.mkEntry(topic3, 1)));
 
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);

Review comment:
       Is there any reason for this change?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       The PR changed how some errors are handled in the `Fetcher`. Do we have any tests for this new behavior?

##########
File path: core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala
##########
@@ -137,11 +137,11 @@ class DelayedFetchTest extends EasyMockSupport {
 
     val fetchStatus = FetchPartitionStatus(
       startOffsetMetadata = LogOffsetMetadata(fetchOffset),
-      fetchInfo = new FetchRequest.PartitionData(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch))
+      fetchInfo = new FetchRequest.PartitionData(topicIds.get("topic"), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch))

Review comment:
       nit: It seems that we could use `TopicIdPartition` directly and remove `topicIds` map entirely. We could also pass the `TopicIdPartition` to `buildFetchMetadata`.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -465,18 +477,18 @@ public void testFetchError() {
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fullFetchResponse(tp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0));
+        client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NOT_LEADER_OR_FOLLOWER, 100L, 0));
         consumerClient.poll(time.timer(0));
         assertTrue(fetcher.hasCompletedFetches());
 
         Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = fetchedRecords();
         assertFalse(partitionRecords.containsKey(tp0));
     }
 
-    private MockClient.RequestMatcher matchesOffset(final TopicPartition tp, final long offset) {
+    private MockClient.RequestMatcher matchesOffset(final TopicIdPartition tp, final long offset) {
         return body -> {
             FetchRequest fetch = (FetchRequest) body;
-            Map<TopicPartition, FetchRequest.PartitionData> fetchData =  fetch.fetchData(topicNames);
+            Map<TopicIdPartition, FetchRequest.PartitionData> fetchData =  fetch.fetchData(topicNames);

Review comment:
       nit: There are two spaces after `=`.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -157,38 +156,38 @@ class FetchSessionTest {
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
 
-    val tp0 = new TopicPartition("foo", 0)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 1)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid()).asJava
+    val tp0 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 0))
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 1))
     val topicNames = topicIds.asScala.map(_.swap).asJava
 
-    def cachedLeaderEpochs(context: FetchContext): Map[TopicPartition, Optional[Integer]] = {
-      val mapBuilder = Map.newBuilder[TopicPartition, Optional[Integer]]
-      context.foreachPartition((tp, _, data) => mapBuilder += tp -> data.currentLeaderEpoch)
+    def cachedLeaderEpochs(context: FetchContext): Map[TopicIdPartition, Optional[Integer]] = {
+      val mapBuilder = Map.newBuilder[TopicIdPartition, Optional[Integer]]
+      context.foreachPartition((tp, data) => mapBuilder += tp -> data.currentLeaderEpoch)
       mapBuilder.result()
     }
 
     val requestData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    requestData1.put(tp0, new FetchRequest.PartitionData(0, 0, 100, Optional.empty()))
-    requestData1.put(tp1, new FetchRequest.PartitionData(10, 0, 100, Optional.of(1)))
-    requestData1.put(tp2, new FetchRequest.PartitionData(10, 0, 100, Optional.of(2)))
+    requestData1.put(tp0.topicPartition, new FetchRequest.PartitionData(topicIds.get("foo"), 0, 0, 100, Optional.empty()))
+    requestData1.put(tp1.topicPartition, new FetchRequest.PartitionData(topicIds.get("foo"), 10, 0, 100, Optional.of(1)))
+    requestData1.put(tp2.topicPartition, new FetchRequest.PartitionData(topicIds.get("bar"), 10, 0, 100, Optional.of(2)))

Review comment:
       nit: We could get the topic id from `tp*.topicId`.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Should we iterate over the partitions in the context to check the `TopicIdPartition`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -222,6 +230,9 @@ private void assignFromUserNoId(Set<TopicPartition> partitions) {
         metadata.update(9, RequestTestUtils.metadataUpdateWithIds("dummy", 1,
             Collections.emptyMap(), singletonMap("noId", 1),
             tp -> validLeaderEpoch, topicIds), false, 0L);
+
+        metadata.fetch().nodes().forEach(node ->
+                apiVersions.update(node.idString(), NodeApiVersions.create()));

Review comment:
       ditto. There is a few other cases in this file.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // run through each partition to resolve them
+    context2.foreachPartition((_, _) => ())

Review comment:
       Should we assert that the `TopicIdPartition` received here contains the topic name?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
+    respData1.put(emptyFoo0, new FetchResponseData.PartitionData()
       .setPartitionIndex(0)
-      .setHighWatermark(100)
-      .setLastStableOffset(100)
-      .setLogStartOffset(100))
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
+    respData1.put(emptyFoo1, new FetchResponseData.PartitionData()
+      .setPartitionIndex(1)
+      .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
     val resp1 = context1.updateAndGenerateResponseData(respData1)
+    // On the latest request version, we should have unknown topic ID errors.
     assertEquals(Errors.NONE, resp1.error())
     assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
+    assertEquals(2, resp1.responseData(topicNames, request1.version).size)
+    resp1.responseData(topicNames, request1.version).forEach( (_, resp) => assertEquals(Errors.UNKNOWN_TOPIC_ID.code, resp.errorCode))
 
-    // Create an incremental fetch request as though no topics changed. However, send a v13 request.
-    // Also simulate the topic ID found on the server.
-    val fooId = Uuid.randomUuid()
-    topicIds.put("foo", fooId)
-    topicNames.put(fooId, "foo")
+    // Create an incremental request where we resolve the partitions
     val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, topicIds, EMPTY_PART_LIST, false)
+    val request2 = createRequest(new JFetchMetadata(resp1.sessionId(), 1), reqData2, EMPTY_PART_LIST, false)
     val context2 = fetchManager.newContext(
       request2.version,
       request2.metadata,
       request2.isFromFollower,
       request2.fetchData(topicNames),
       request2.forgottenTopics(topicNames),
-      topicIds
+      topicNames
     )
-
-    assertEquals(classOf[SessionErrorContext], context2.getClass)
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
-      context2.updateAndGenerateResponseData(respData2).error())
+    // run through each partition to resolve them

Review comment:
       nit: I would expand this comment a little and stress the fact that topic names are lazily resolved when the partitions are iterated over.

##########
File path: core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
##########
@@ -1091,18 +1089,20 @@ class AbstractFetcherThreadTest {
 
     override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
       val fetchData = mutable.Map.empty[TopicPartition, FetchRequest.PartitionData]
-      partitionMap.foreach { case (partition, state) =>
+      partitionMap.foreach { case (partition, state) => 0
+        .equals(0)

Review comment:
       `0.equals(0)` was very likely put here by mistake.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava

Review comment:
       It seems to be that it would be simpler to declare `fooId` and `barId` and to use them instead of getting them from the map.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       Should we add any tests for the new logic in KafkaApis?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       I wonder if we should add a third topic which is never resolved. What do you think?

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -1361,102 +1542,113 @@ class FetchSessionTest {
     val resp4 = context2.updateAndGenerateResponseData(respData)
     assertEquals(Errors.NONE, resp4.error)
     assertEquals(resp1.sessionId, resp4.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp4.responseData(topicNames, request2.version).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
   }
 
   @Test
   def testDeprioritizesPartitionsWithRecordsOnly(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val tp1 = new TopicPartition("foo", 1)
-    val tp2 = new TopicPartition("bar", 2)
-    val tp3 = new TopicPartition("zar", 3)
     val topicIds = Map("foo" -> Uuid.randomUuid(), "bar" -> Uuid.randomUuid(), "zar" -> Uuid.randomUuid()).asJava
     val topicNames = topicIds.asScala.map(_.swap).asJava
+    val tp1 = new TopicIdPartition(topicIds.get("foo"), new TopicPartition("foo", 1))
+    val tp2 = new TopicIdPartition(topicIds.get("bar"), new TopicPartition("bar", 2))
+    val tp3 = new TopicIdPartition(topicIds.get("zar"), new TopicPartition("zar", 3))
 
-    val reqData = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData.put(tp1, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp2, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
-    reqData.put(tp3, new FetchRequest.PartitionData(100, 0, 1000, Optional.of(5), Optional.of(4)))
+    val reqData = new util.LinkedHashMap[TopicIdPartition, FetchRequest.PartitionData]
+    reqData.put(tp1, new FetchRequest.PartitionData(topicIds.get("foo"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp2, new FetchRequest.PartitionData(topicIds.get("bar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
+    reqData.put(tp3, new FetchRequest.PartitionData(topicIds.get("zar"), 100, 0, 1000, Optional.of(5), Optional.of(4)))
 
     // Full fetch context returns all partitions in the response
     val context1 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), JFetchMetadata.INITIAL, false,
-     reqData, Collections.emptyList(), topicIds)
+     reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[FullFetchContext], context1.getClass)
 
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData1.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData1.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
 
     val resp1 = context1.updateAndGenerateResponseData(respData1)
     assertEquals(Errors.NONE, resp1.error)
     assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2, tp3), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
 
     // Incremental fetch context returns partitions with changes but only deprioritizes
     // the partitions with records
     val context2 = fetchManager.newContext(ApiKeys.FETCH.latestVersion(), new JFetchMetadata(resp1.sessionId, 1), false,
-      reqData, Collections.emptyList(), topicIds)
+      reqData, Collections.emptyList(), topicNames)
     assertEquals(classOf[IncrementalFetchContext], context2.getClass)
 
     // Partitions are ordered in the session as per last response
     assertPartitionsOrder(context2, Seq(tp1, tp2, tp3))
 
     // Response is empty
-    val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     val resp2 = context2.updateAndGenerateResponseData(respData2)
     assertEquals(Errors.NONE, resp2.error)
     assertEquals(resp1.sessionId, resp2.sessionId)
     assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // All partitions with changes should be returned.
-    val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
+    val respData3 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
     respData3.put(tp1, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp1.partition)
+      .setPartitionIndex(tp1.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     respData3.put(tp2, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp2.partition)
+      .setPartitionIndex(tp2.topicPartition.partition)
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
       .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
-      .setPartitionIndex(tp3.partition)
+      .setPartitionIndex(tp3.topicPartition.partition)
       .setHighWatermark(50)
       .setLastStableOffset(50)
       .setLogStartOffset(0))
     val resp3 = context2.updateAndGenerateResponseData(respData3)
     assertEquals(Errors.NONE, resp3.error)
     assertEquals(resp1.sessionId, resp3.sessionId)
-    assertEquals(Utils.mkSet(tp1, tp2), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
+    assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
 
     // Only the partitions whose returned records in the last response
     // were deprioritized
     assertPartitionsOrder(context2, Seq(tp1, tp3, tp2))
   }
 
-  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicPartition]): Unit = {
-    val partitionsInContext = ArrayBuffer.empty[TopicPartition]
-    context.foreachPartition { (tp, _, _) =>
+  private def assertPartitionsOrder(context: FetchContext, partitions: Seq[TopicIdPartition]): Unit = {
+    val partitionsInContext = ArrayBuffer.empty[TopicIdPartition]
+    context.foreachPartition { (tp, _) =>
       partitionsInContext += tp
     }
     assertEquals(partitions, partitionsInContext.toSeq)
   }
 }
+
+object FetchSessionTest {
+  def idUsageCombinations: java.util.stream.Stream[Arguments] = {
+    Seq(
+      Arguments.of(true, true),
+      Arguments.of(true, false),
+      Arguments.of(false, true),
+      Arguments.of(false, false)
+    ).asJava.stream()
+  }
+}

Review comment:
       I wonder if we could add a few more unit tests. For instance, we should test the equals/hash methods of the CachedPartition (and possibly other methods there). We might want to add some for other classes as well. What do you think?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));

Review comment:
       I think that the grouping is slower because it has to allocate another Map, Sets for each Uuid, etc.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       I think that would for instance append when the controller fails over to an older IBP during an upgrade. This should remove the topic ids which means that v12 will be used for the next fetch request and trigger a FETCH_SESSION_TOPIC_ID_ERROR. In this particular case, re-trying directly would be the optimal way to proceed for a follower. I wonder if they are other cases to consider here.
   
   For the consumer, it is definitely different.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +173,35 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int = if (topicId != Uuid.ZERO_UUID) (31 * partition) + topicId.hashCode else
+    (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
 
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   *
+   * This is because when we use topic IDs, a partition with a given ID and an unknown name is the same as a partition with that
+   * ID and a known name. This means we can only use topic ID and partition when determining equality.
+   *
+   * On the other hand, if we are using topic names, all IDs are zero. This means we can only use topic name and partition
+   * when determining equality.
+   */
   override def equals(that: Any): Boolean =
     that match {
       case that: CachedPartition =>
         this.eq(that) ||
           (that.canEqual(this) &&

Review comment:
       Right. It seems to be that the `canEqual(this)` does not make any sense here. Could you double check?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -708,40 +701,41 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
-    val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
-    val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
-    val sessionTopicIds = mutable.Map[String, Uuid]()
+    val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
+    val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
     if (fetchRequest.isFromFollower) {
       // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
       if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
-        fetchContext.foreachPartition { (topicPartition, topicId, data) =>
-          sessionTopicIds.put(topicPartition.topic(), topicId)
-          if (!metadataCache.contains(topicPartition))
-            erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
+        fetchContext.foreachPartition { (topicIdPartition, data) =>
+          if (topicIdPartition.topicPartition.topic == null )
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
+          else if (!metadataCache.contains(topicIdPartition.topicPartition))
+            erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
           else
-            interesting += (topicPartition -> data)
+            interesting += (topicIdPartition -> data)
         }
       } else {
-        fetchContext.foreachPartition { (part, topicId, _) =>
-          sessionTopicIds.put(part.topic(), topicId)
-          erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
+        fetchContext.foreachPartition { (topicIdPartition, _) =>
+          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)

Review comment:
       I guess that it does not change much in the end. I was considering this in order to be consistent with how we handle this for the consumer.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -66,16 +69,28 @@ public PartitionData(
             int maxBytes,
             Optional<Integer> currentLeaderEpoch
         ) {
-            this(fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());
+            this(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, Optional.empty());

Review comment:
       Yeah, that's a good question. I guess that that constructor is convenient for tests but might be bug prone in the regular code. I am tempted to remove it entirely.... What do you think?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;

Review comment:
       Sorry, I meant below assertions not above. Yes, it seems that they are testing the logic of the `FetchRequest` itself and not really the logic of the FetchSessionHandler.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());

Review comment:
       Correct. I was referring to the upgrade case. We might need to handle the downgrade case for https://github.com/apache/kafka/pull/11459.

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -428,82 +442,165 @@ public void testIdUsageRevokedOnIdDowngrade() {
         List<Integer> partitions = Arrays.asList(0, 1);
         partitions.forEach(partition -> {
             String testType = partition == 0 ? "updating a partition" : "adding a new partition";
-            Map<String, Uuid> topicIds = Collections.singletonMap("foo", Uuid.randomUuid());
+            Uuid fooId = Uuid.randomUuid();
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
-            builder.add(new TopicPartition("foo", 0),  topicIds.get("foo"),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+            builder.add(new TopicPartition("foo", 0),
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
                     data.toSend(), data.sessionPartitions());
             assertTrue(data.metadata().isFull());
             assertTrue(data.canUseTopicIds());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                    respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
+                    respMap(new RespEntry("foo", 0, fooId, 10, 20)));
             handler.handleResponse(resp, ApiKeys.FETCH.latestVersion());
 
             // Try to remove a topic ID from an existing topic partition (0) or add a new topic partition (1) without an ID.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
-            builder2.add(new TopicPartition("foo", partition), Uuid.ZERO_UUID,
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            builder2.add(new TopicPartition("foo", partition),
+                    new FetchRequest.PartitionData(Uuid.ZERO_UUID, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data2 = builder2.build();
-            // Should have the same session ID and next epoch, but can no longer use topic IDs.
-            // The receiving broker will close the session if we were previously using topic IDs.
+            // Should have the same session ID, and next epoch and can no longer use topic IDs.
+            // The receiving broker will handle closing the session.
             assertEquals(123, data2.metadata().sessionId(), "Did not use same session when " + testType);
             assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch when " + testType);
             assertFalse(data2.canUseTopicIds());
         });
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTopicIdReplaced(boolean fetchRequestUsesIds) {
+        TopicPartition tp = new TopicPartition("foo", 0);
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        Uuid topicId1 = Uuid.randomUuid();
+        builder.add(tp,
+                new FetchRequest.PartitionData(topicId1, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId1, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertTrue(data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, Uuid.ZERO_UUID, 10, 20)));
+        handler.handleResponse(resp, (short) 12);
+
+        // Try to add a new topic ID.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        Uuid topicId2 = Uuid.randomUuid();
+        // Use the same data besides the topic ID.
+        FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(topicId2, 0, 100, 200, Optional.empty());
+        builder2.add(tp, partitionData);
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // The old topic ID partition should be in toReplace, and the new one should be in toSend.
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId2, 0, 0, 100, 200)),
+                data2.toSend(), data2.sessionPartitions());
+        assertEquals(Collections.singletonList(new TopicIdPartition(topicId1, tp)), data2.toReplace());
+        // Should have the same session ID, and next epoch and can use topic IDs.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have correct epoch");
+        assertTrue(data2.canUseTopicIds());
+
+        short version = fetchRequestUsesIds ? ApiKeys.FETCH.latestVersion() : 12;
+        FetchRequest fetchRequest = FetchRequest.Builder
+                .forReplica(version, 0, 1, 1, data2.toSend())
+                .removed(data2.toForget())
+                .replaced(data2.toReplace())
+                .metadata(data2.metadata()).build(version);
+
+        assertEquals(fetchRequestUsesIds, fetchRequest.data().forgottenTopicsData().size() > 0);
+        assertEquals(1, fetchRequest.data().topics().size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testSessionEpochWhenMixedUsageOfTopicIDs(boolean startsWithTopicIds) {
+        Uuid fooId = startsWithTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
+        Uuid barId = startsWithTopicIds ? Uuid.ZERO_UUID : Uuid.randomUuid();
+        short responseVersion = startsWithTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
+
+        TopicPartition tp0 = new TopicPartition("foo", 0);
+        TopicPartition tp1 = new TopicPartition("bar", 1);
+
+        FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
+        FetchSessionHandler.Builder builder = handler.newBuilder();
+        builder.add(tp0,
+                new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data = builder.build();
+        assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200)),
+                data.toSend(), data.sessionPartitions());
+        assertTrue(data.metadata().isFull());
+        assertEquals(startsWithTopicIds, data.canUseTopicIds());
+
+        FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
+                respMap(new RespEntry("foo", 0, fooId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
+
+        // Re-add the first partition. Then add a partition with opposite ID usage.
+        FetchSessionHandler.Builder builder2 = handler.newBuilder();
+        builder2.add(tp0,
+                new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
+        builder2.add(tp1,
+                new FetchRequest.PartitionData(barId, 0, 100, 200, Optional.empty()));
+        FetchSessionHandler.FetchRequestData data2 = builder2.build();
+        // Should have the same session ID, and the next epoch and can not use topic IDs.
+        // The receiving broker will handle closing the session.
+        assertEquals(123, data2.metadata().sessionId(), "Did not use same session");
+        assertEquals(1, data2.metadata().epoch(), "Did not have final epoch");
+        assertFalse(data2.canUseTopicIds());
+    }
+
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testIdUsageWithAllForgottenPartitions(boolean useTopicIds) {
         // We want to test when all topics are removed from the session
         Uuid topicId = useTopicIds ? Uuid.randomUuid() : Uuid.ZERO_UUID;
-        Short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
-        Map<String, Uuid> topicIds = Collections.singletonMap("foo", topicId);
+        short responseVersion = useTopicIds ? ApiKeys.FETCH.latestVersion() : 12;
         FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
 
         // Add topic foo to the session
         FetchSessionHandler.Builder builder = handler.newBuilder();
-        builder.add(new TopicPartition("foo", 0), topicIds.get("foo"),
-                new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
+        builder.add(new TopicPartition("foo", 0),
+                new FetchRequest.PartitionData(topicId, 0, 100, 200, Optional.empty()));
         FetchSessionHandler.FetchRequestData data = builder.build();
-        assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200)),
+        assertMapsEqual(reqMap(new ReqEntry("foo", topicId, 0, 0, 100, 200)),
                 data.toSend(), data.sessionPartitions());
         assertTrue(data.metadata().isFull());
         assertEquals(useTopicIds, data.canUseTopicIds());
 
         FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20)), topicIds);
-        handler.handleResponse(resp, responseVersion.shortValue());
+                respMap(new RespEntry("foo", 0, topicId, 10, 20)));
+        handler.handleResponse(resp, responseVersion);
 
         // Remove the topic from the session
         FetchSessionHandler.Builder builder2 = handler.newBuilder();
         FetchSessionHandler.FetchRequestData data2 = builder2.build();
-        // Should have the same session ID and next epoch, but can no longer use topic IDs.
-        // The receiving broker will close the session if we were previously using topic IDs.
+        // Should have the same session ID, next epoch, and same ID usage.
         assertEquals(123, data2.metadata().sessionId(), "Did not use same session when useTopicIds was " + useTopicIds);

Review comment:
       Yeah, it would be good to assert what we expect in `data2` for completeness.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
##########
@@ -4814,6 +4842,7 @@ private void buildDependencies(MetricConfig metricConfig,
         metadata = new ConsumerMetadata(0, metadataExpireMs, false, false,
                 subscriptions, logContext, new ClusterResourceListeners());
         client = new MockClient(time, metadata);
+        client.setNodeApiVersions(NodeApiVersions.create());
         metrics = new Metrics(metricConfig, time);
         consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time,
                 100, 1000, Integer.MAX_VALUE);

Review comment:
       Yes, I was referring to those. Ack, I missed them during my first read.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava
+    val topicIds = topicNames.asScala.map(_.swap).asJava
+    val foo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 0))
+    val foo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition("foo", 1))
+    val emptyFoo0 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 0))
+    val emptyFoo1 = new TopicIdPartition(topicIds.getOrDefault("foo", Uuid.ZERO_UUID), new TopicPartition(null, 1))
 
-    // Create a new fetch session with foo-0
+    // Create a new fetch session with foo-0 and foo-1
     val reqData1 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
-    reqData1.put(new TopicPartition("foo", 0), new FetchRequest.PartitionData(0, 0, 100,
+    reqData1.put(foo0.topicPartition, new FetchRequest.PartitionData(foo0.topicId, 0, 0, 100,
       Optional.empty()))
-    val request1 = createRequestWithoutTopicIds(JFetchMetadata.INITIAL, reqData1, topicIds, EMPTY_PART_LIST, false)
-    // Start a fetch session using a request version that does not use topic IDs.
+    reqData1.put(foo1.topicPartition, new FetchRequest.PartitionData(foo1.topicId,10, 0, 100,
+      Optional.empty()))
+    val request1 = createRequest(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
+    // Simulate unknown topic ID for foo.
+    val topicNamesOnlyBar = Collections.singletonMap(topicIds.get("bar"), "bar")
+    // We should not throw error since we have an older request version.
     val context1 = fetchManager.newContext(
       request1.version,
       request1.metadata,
       request1.isFromFollower,
-      request1.fetchData(topicNames),
-      request1.forgottenTopics(topicNames),
-      topicIds
+      request1.fetchData(topicNamesOnlyBar),
+      request1.forgottenTopics(topicNamesOnlyBar),
+      topicNamesOnlyBar
     )
     assertEquals(classOf[FullFetchContext], context1.getClass)
-    val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
-    respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
+    val respData1 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]

Review comment:
       Yeah, I meant exactly that. How about using `assertPartitionsOrder` helper? The assertion would be more complete.

##########
File path: core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
##########
@@ -659,88 +670,108 @@ class FetchSessionTest {
   }
 
   @Test
-  def testIncrementalFetchSessionWithIdsWhenSessionDoesNotUseIds() : Unit = {
+  def testFetchSessionWithUnknownId(): Unit = {
     val time = new MockTime()
     val cache = new FetchSessionCache(10, 1000)
     val fetchManager = new FetchManager(time, cache)
-    val topicIds = new util.HashMap[String, Uuid]()
-    val topicNames = new util.HashMap[Uuid, String]()
+    val topicNames = Map(Uuid.randomUuid() -> "foo", Uuid.randomUuid() -> "bar").asJava

Review comment:
       You could use `assertPartitionsOrder` helper here as well.

##########
File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
##########
@@ -3530,37 +3534,37 @@ class KafkaApisTest {
   def testSizeOfThrottledPartitions(): Unit = {
     val topicNames = new util.HashMap[Uuid, String]
     val topicIds = new util.HashMap[String, Uuid]()
-    def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
-      val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
+    def fetchResponse(data: Map[TopicIdPartition, String]): FetchResponse = {
+      val responseData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData](
         data.map { case (tp, raw) =>
           tp -> new FetchResponseData.PartitionData()
-            .setPartitionIndex(tp.partition)
+            .setPartitionIndex(tp.topicPartition.partition)
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
             .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
-        val id = Uuid.randomUuid()
-        topicIds.put(tp.topic(), id)
-        topicNames.put(id, tp.topic())
+        topicIds.put(tp.topicPartition.topic, tp.topicId)
+        topicNames.put(tp.topicId, tp.topicPartition.topic)
       }
-      FetchResponse.of(Errors.NONE, 100, 100, responseData, topicIds)
+      FetchResponse.of(Errors.NONE, 100, 100, responseData)
     }
 
-    val throttledPartition = new TopicPartition("throttledData", 0)
+    val throttledPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("throttledData", 0))
     val throttledData = Map(throttledPartition -> "throttledData")
     val expectedSize = FetchResponse.sizeOf(FetchResponseData.HIGHEST_SUPPORTED_VERSION,
-      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.iterator, topicIds)
+      fetchResponse(throttledData).responseData(topicNames, FetchResponseData.HIGHEST_SUPPORTED_VERSION).entrySet.asScala.map( entry =>
+      (new TopicIdPartition(Uuid.ZERO_UUID, entry.getKey), entry.getValue)).toMap.asJava.entrySet.iterator)
 
-    val response = fetchResponse(throttledData ++ Map(new TopicPartition("nonThrottledData", 0) -> "nonThrottledData"))
+    val response = fetchResponse(throttledData ++ Map(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("nonThrottledData", 0)) -> "nonThrottledData"))
 
     val quota = Mockito.mock(classOf[ReplicationQuotaManager])
     Mockito.when(quota.isThrottled(ArgumentMatchers.any(classOf[TopicPartition])))
-      .thenAnswer(invocation => throttledPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
+      .thenAnswer(invocation => throttledPartition.topicPartition == invocation.getArgument(0).asInstanceOf[TopicPartition])
 
-    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota, topicIds))
+    assertEquals(expectedSize, KafkaApis.sizeOfThrottledPartitions(FetchResponseData.HIGHEST_SUPPORTED_VERSION, response, quota))
   }
 
   @Test

Review comment:
       That is right.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Should we add or extend a test in `FetcherTest` to cover this change? I would like to have one which ensure that the request sent is populated correctly (especially the replaced part) by the fetcher based on the session handler. It seems that we don't have such test in the suite at the moment.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -199,26 +235,31 @@ public FetchRequest build(short version) {
             fetchRequestData.setMaxBytes(maxBytes);
             fetchRequestData.setIsolationLevel(isolationLevel.id());
             fetchRequestData.setForgottenTopicsData(new ArrayList<>());
-            toForget.stream()
-                .collect(Collectors.groupingBy(TopicPartition::topic, LinkedHashMap::new, Collectors.toList()))
-                .forEach((topic, partitions) ->
-                    fetchRequestData.forgottenTopicsData().add(new FetchRequestData.ForgottenTopic()
-                        .setTopic(topic)
-                        .setTopicId(topicIds.getOrDefault(topic, Uuid.ZERO_UUID))
-                        .setPartitions(partitions.stream().map(TopicPartition::partition).collect(Collectors.toList())))
-                );
-            fetchRequestData.setTopics(new ArrayList<>());
+
+            Map<String, FetchRequestData.ForgottenTopic> forgottenTopicMap = new LinkedHashMap<>();
+            addToForgottenTopicMap(removed, forgottenTopicMap);
+
+            // If a version older than v13 is used, topic-partition which were replaced
+            // by a topic-partition with the same name but a different topic ID are not
+            // sent out in the "forget" set in order to not remove the newly added
+            // partition in the "fetch" set.
+            if (version >= 13) {
+                addToForgottenTopicMap(replaced, forgottenTopicMap);
+            }

Review comment:
       Should we add a few unit tests to validate the changes that we have done in this class? We could add a few to FetchRequestTest (not use if it already exists though).

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
##########
@@ -314,8 +356,7 @@ public int maxBytes() {
 
     // For versions < 13, builds the partitionData map using only the FetchRequestData.
     // For versions 13+, builds the partitionData map using both the FetchRequestData and a mapping of topic IDs to names.
-    // Throws UnknownTopicIdException for versions 13+ if the topic ID was unknown to the server.
-    public Map<TopicPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {
+    public Map<TopicIdPartition, PartitionData> fetchData(Map<Uuid, String> topicNames) throws UnknownTopicIdException {

Review comment:
       Do we have a unit test for this one and for `forgottenTopics`?

##########
File path: clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java
##########
@@ -242,65 +244,68 @@ public void testIncrementals() {
             FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
             FetchSessionHandler.Builder builder = handler.newBuilder();
             addTopicId(topicIds, topicNames, "foo", version);
-            builder.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 110, 210, Optional.empty()));
+            Uuid fooId = topicIds.getOrDefault("foo", Uuid.ZERO_UUID);
+            TopicPartition foo0 = new TopicPartition("foo", 0);
+            TopicPartition foo1 = new TopicPartition("foo", 1);
+            builder.add(foo0, new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));
+            builder.add(foo1, new FetchRequest.PartitionData(fooId, 10, 110, 210, Optional.empty()));
             FetchSessionHandler.FetchRequestData data = builder.build();
-            assertMapsEqual(reqMap(new ReqEntry("foo", 0, 0, 100, 200),
-                    new ReqEntry("foo", 1, 10, 110, 210)),
+            assertMapsEqual(reqMap(new ReqEntry("foo", fooId, 0, 0, 100, 200),
+                    new ReqEntry("foo", fooId, 1, 10, 110, 210)),
                     data.toSend(), data.sessionPartitions());
             assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
             assertEquals(INITIAL_EPOCH, data.metadata().epoch());
 
             FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
-                respMap(new RespEntry("foo", 0, 10, 20),
-                        new RespEntry("foo", 1, 10, 20)), topicIds);
+                respMap(new RespEntry("foo", 0, fooId, 10, 20),
+                        new RespEntry("foo", 1, fooId, 10, 20)));
             handler.handleResponse(resp, version);
 
             // Test an incremental fetch request which adds one partition and modifies another.
             FetchSessionHandler.Builder builder2 = handler.newBuilder();
             addTopicId(topicIds, topicNames, "bar", version);
-            builder2.add(new TopicPartition("foo", 0), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(0, 100, 200, Optional.empty()));
-            builder2.add(new TopicPartition("foo", 1), topicIds.getOrDefault("foo", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(10, 120, 210, Optional.empty()));
-            builder2.add(new TopicPartition("bar", 0), topicIds.getOrDefault("bar", Uuid.ZERO_UUID),
-                    new FetchRequest.PartitionData(20, 200, 200, Optional.empty()));
+            Uuid barId = topicIds.getOrDefault("bar", Uuid.ZERO_UUID);
+            TopicPartition bar0 = new TopicPartition("bar", 0);
+            builder2.add(foo0,
+                    new FetchRequest.PartitionData(fooId, 0, 100, 200, Optional.empty()));

Review comment:
       There are a few more cases where we could put the partition data back on the previous line in this file.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       I think that would for instance happen when the controller fails over to an older IBP during an upgrade. This should remove the topic ids which means that v12 will be used for the next fetch request and trigger a FETCH_SESSION_TOPIC_ID_ERROR. In this particular case, re-trying directly would be the optimal way to proceed for a follower. I wonder if they are other cases to consider here.
   
   For the consumer, it is definitely different.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Sorry, I wanted to say happen.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -374,7 +374,7 @@ abstract class AbstractFetcherThread(name: String,
                       }
                     }
                   } catch {
-                    case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
+                    case ime@(_: CorruptRecordException | _: InvalidRecordException) =>

Review comment:
       Anyway, we don't need to address this in this PR. I just wanted to point out that there is an opportunity for an improvement.

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -413,8 +413,20 @@ abstract class AbstractFetcherThread(name: String,
 
                 case Errors.UNKNOWN_TOPIC_OR_PARTITION =>
                   warn(s"Received ${Errors.UNKNOWN_TOPIC_OR_PARTITION} from the leader for partition $topicPartition. " +
-                       "This error may be returned transiently when the partition is being created or deleted, but it is not " +
-                       "expected to persist.")
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.UNKNOWN_TOPIC_ID =>
+                  warn(s"Received ${Errors.UNKNOWN_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")
+                  partitionsWithError += topicPartition
+
+                case Errors.INCONSISTENT_TOPIC_ID =>
+                  warn(s"Received ${Errors.INCONSISTENT_TOPIC_ID} from the leader for partition $topicPartition. " +
+                    "This error may be returned transiently when the partition is being created or deleted, but it is not " +
+                    "expected to persist.")

Review comment:
       Do we have unit tests covering those cases? There are almost no changes in `AbstractFetcherThreadTest` so it seems that we don't. Are they somewhere else perhaps?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -163,18 +178,37 @@ class CachedPartition(val topic: String,
     mustRespond
   }
 
-  override def hashCode: Int = Objects.hash(new TopicPartition(topic, partition), topicId)
+  /**
+   * We have different equality checks depending on whether topic IDs are used.
+   * This means we need a different hash function as well. We use name to calculate the hash if the ID is zero and unused.
+   * Otherwise, we use the topic ID in the hash calculation.
+   *
+   * @return the hash code for the CachedPartition depending on what request version we are using.
+   */
+  override def hashCode: Int =
+    if (topicId != Uuid.ZERO_UUID)
+      (31 * partition) + topicId.hashCode
+    else
+      (31 * partition) + topic.hashCode
 
   def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]

Review comment:
       I guess that we could remove it now.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       nit: Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover them as well?

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Should we use the same name for both `maybeSetUnknownName` and `maybeResolveUnknownName`? I guess that you could differ by their argument.
   
   If we add unit tests for other methods of this class, should we cover all the methods that we have changed or added as well?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -305,9 +304,10 @@ class ReplicaFetcherThread(name: String,
     } else {
       val version: Short = if (fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else fetchRequestVersion
       val requestBuilder = FetchRequest.Builder
-        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend, fetchData.topicIds)
+        .forReplica(version, replicaId, maxWait, minBytes, fetchData.toSend)
         .setMaxBytes(maxBytes)
-        .toForget(fetchData.toForget)
+        .removed(fetchData.toForget)
+        .replaced(fetchData.toReplace)

Review comment:
       Do we have tests verifying this change?

##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
-                    sessionTopicNames = new HashMap<>();
+                    sessionTopicNames = Collections.emptyMap();
                 }
-                topicIds = null;
                 Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {
+                        // Re-add the replaced partition to the end of 'next'
+                        next.put(topicPartition, nextData);
+                        entry.setValue(nextData);
+                        replaced.add(new TopicIdPartition(prevData.topicId, topicPartition));
+                    } else if (!prevData.equals(nextData)) {
                         // Re-add the altered partition to the end of 'next'
                         next.put(topicPartition, nextData);
                         entry.setValue(nextData);
-                        altered.add(topicPartition);
+                        altered.add(new TopicIdPartition(nextData.topicId, topicPartition));
                     }
                 } else {
                     // Remove this partition from the session.
                     iter.remove();
                     // Indicate that we no longer want to listen to this partition.
-                    removed.add(topicPartition);
+                    removed.add(new TopicIdPartition(prevData.topicId, topicPartition));
                     // If we do not have this topic ID in the builder or the session, we can not use topic IDs.
-                    if (canUseTopicIds && !topicIds.containsKey(topicPartition.topic()) && !sessionTopicIds.containsKey(topicPartition.topic()))
+                    if (canUseTopicIds && prevData.topicId == Uuid.ZERO_UUID)

Review comment:
       Should we use `equals` instead of `==`? We use `equals` at L304 btw. 

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       We should have a test in the Fetcher which ensure that the builder received the correct information. Then we could have one for the request which ensure that the builder does its job correctly as well.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -262,11 +262,12 @@ public synchronized int sendFetches() {
                 maxVersion = ApiKeys.FETCH.latestVersion();
             }
             final FetchRequest.Builder request = FetchRequest.Builder
-                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend(), data.topicIds())
+                    .forConsumer(maxVersion, this.maxWaitMs, this.minBytes, data.toSend())
                     .isolationLevel(isolationLevel)
                     .setMaxBytes(this.maxBytes)
                     .metadata(data.metadata())
-                    .toForget(data.toForget())
+                    .removed(data.toForget())
+                    .replaced(data.toReplace())

Review comment:
       Right. You might have to assert on the request in the fetcher as well. As you said, we can't really get the data out from the builder otherwise.

##########
File path: core/src/main/scala/kafka/server/FetchSession.scala
##########
@@ -93,27 +93,42 @@ class CachedPartition(val topic: String,
   def this(topic: String, partition: Int, topicId: Uuid) =
     this(topic, topicId, partition, -1, -1, -1, Optional.empty(), -1, -1, Optional.empty[Integer])
 
-  def this(part: TopicPartition, topicId: Uuid) =
-    this(part.topic, part.partition, topicId)
+  def this(part: TopicIdPartition) = {
+    this(part.topic, part.partition, part.topicId)
+  }
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData) =
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, -1,
       reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
 
-  def this(part: TopicPartition, id: Uuid, reqData: FetchRequest.PartitionData,
+  def this(part: TopicIdPartition, reqData: FetchRequest.PartitionData,
            respData: FetchResponseData.PartitionData) =
-    this(part.topic, id, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
+    this(part.topic, part.topicId, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
       reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
 
-  def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
+  def reqData = new FetchRequest.PartitionData(topicId, fetchOffset, fetcherLogStartOffset, maxBytes, leaderEpoch, lastFetchedEpoch)
 
-  def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = {
+  def maybeUpdateRequestParamsOrName(reqData: FetchRequest.PartitionData, name: String): Unit = {
     // Update our cached request parameters.
     maxBytes = reqData.maxBytes
     fetchOffset = reqData.fetchOffset
     fetcherLogStartOffset = reqData.logStartOffset
     leaderEpoch = reqData.currentLeaderEpoch
     lastFetchedEpoch = reqData.lastFetchedEpoch
+    // Update name if needed
+    maybeSetUnknownName(name)
+  }
+
+  def maybeSetUnknownName(name: String): Unit = {
+    if (this.topic == null) {
+      this.topic = name
+    }
+  }
+
+  def maybeResolveUnknownName(topicNames: FetchSession.TOPIC_NAME_MAP): Unit = {

Review comment:
       Yeah, I agree with you. Perhaps, we could just remove the maybeSetTopicName and move its logic into the update request params method.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jolshan commented on a change in pull request #11331: KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java
##########
@@ -285,52 +268,57 @@ public FetchRequestData build() {
             if (nextMetadata.isFull()) {
                 if (log.isDebugEnabled()) {
                     log.debug("Built full fetch {} for node {} with {}.",
-                              nextMetadata, node, partitionsToLogString(next.keySet()));
+                            nextMetadata, node, topicPartitionsToLogString(next.keySet()));
                 }
                 sessionPartitions = next;
                 next = null;
+                Map<TopicPartition, PartitionData> toSend =
+                        Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
                 // Only add topic IDs to the session if we are using topic IDs.
                 if (canUseTopicIds) {
-                    sessionTopicIds = topicIds;
-                    sessionTopicNames = new HashMap<>(topicIds.size());
-                    topicIds.forEach((name, id) -> sessionTopicNames.put(id, name));
+                    Map<Uuid, Set<String>> newTopicNames = sessionPartitions.entrySet().stream().collect(Collectors.groupingByConcurrent(entry -> entry.getValue().topicId,
+                            Collectors.mapping(entry -> entry.getKey().topic(), Collectors.toSet())));
+
+                    sessionTopicNames = new HashMap<>(newTopicNames.size());
+                    // There should only be one topic name per topic ID.
+                    newTopicNames.forEach((topicId, topicNamesSet) -> topicNamesSet.forEach(topicName -> sessionTopicNames.put(topicId, topicName)));
                 } else {
-                    sessionTopicIds = new HashMap<>();
                     sessionTopicNames = new HashMap<>();
                 }
-                topicIds = null;
-                Map<TopicPartition, PartitionData> toSend =
-                    Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
-                Map<String, Uuid> toSendTopicIds =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicIds));
-                Map<Uuid, String> toSendTopicNames =
-                    Collections.unmodifiableMap(new HashMap<>(sessionTopicNames));
-                return new FetchRequestData(toSend, Collections.emptyList(), toSend, toSendTopicIds, toSendTopicNames, nextMetadata, canUseTopicIds);
+                return new FetchRequestData(toSend, Collections.emptyList(), Collections.emptyList(), toSend, nextMetadata, canUseTopicIds);
             }
 
-            List<TopicPartition> added = new ArrayList<>();
-            List<TopicPartition> removed = new ArrayList<>();
-            List<TopicPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> added = new ArrayList<>();
+            List<TopicIdPartition> removed = new ArrayList<>();
+            List<TopicIdPartition> altered = new ArrayList<>();
+            List<TopicIdPartition> replaced = new ArrayList<>();
             for (Iterator<Entry<TopicPartition, PartitionData>> iter =
-                     sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
+                 sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
                 Entry<TopicPartition, PartitionData> entry = iter.next();
                 TopicPartition topicPartition = entry.getKey();
                 PartitionData prevData = entry.getValue();
                 PartitionData nextData = next.remove(topicPartition);
                 if (nextData != null) {
-                    if (!prevData.equals(nextData)) {
+                    // We basically check if the new partition had the same topic ID. If not,
+                    // we add it to the "replaced" set.
+                    if (!prevData.topicId.equals(nextData.topicId) && !prevData.topicId.equals(Uuid.ZERO_UUID)) {

Review comment:
       Do we not care to change IDs if the data is equal? We wouldn't usually send a request and I don't know if it is possible to even have the same data in such a case.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org