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-unsubscr...@kafka.apache.org

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


Reply via email to