cychiu8 commented on code in PR #22252:
URL: https://github.com/apache/kafka/pull/22252#discussion_r3234885860


##########
server/src/test/java/org/apache/kafka/server/requests/FetchRequestTest.java:
##########
@@ -0,0 +1,736 @@
+/*
+ * 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.server.requests;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.FetchRequestData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.FetchMetadata;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.test.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestDefaults;
+import org.apache.kafka.common.test.api.Type;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
+import org.apache.kafka.server.IntegrationTestUtils;
+import org.apache.kafka.server.TestUtils;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.apache.kafka.server.TestUtils.consumeRecords;
+import static 
org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG;
+import static org.apache.kafka.server.config.ServerConfigs.BROKER_RACK_CONFIG;
+import static 
org.apache.kafka.server.config.ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG;
+import static 
org.apache.kafka.server.config.ServerLogConfigs.NUM_PARTITIONS_CONFIG;
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+@ClusterTestDefaults(
+    types = {Type.KRAFT},
+    brokers = FetchRequestTest.BROKER_COUNT,
+    serverProperties = {
+        @ClusterConfigProperty(id = 0, key = BROKER_RACK_CONFIG, value = "0"),
+        @ClusterConfigProperty(id = 1, key = BROKER_RACK_CONFIG, value = "1"),
+        @ClusterConfigProperty(key = REPLICA_SELECTOR_CLASS_CONFIG,
+            value = 
"org.apache.kafka.common.replica.RackAwareReplicaSelector"),
+        @ClusterConfigProperty(key = CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value 
= "false"),
+        @ClusterConfigProperty(key = NUM_PARTITIONS_CONFIG, value = "1"),
+        @ClusterConfigProperty(key = OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 
value = "2"),
+    }
+)
+public class FetchRequestTest {
+
+    public static final int BROKER_COUNT = 2;
+    public static final String TOPIC = "test-fetch-from-follower";
+    public static final int LEADER_BROKER_ID = 0;
+    public static final int FOLLOWER_BROKER_ID = 1;
+
+    private ClusterInstance cluster;
+    Map<String, Uuid> topicIds;
+
+    @ClusterTest
+    @Timeout(15)
+    public void 
testFollowerCompleteDelayedFetchesOnReplication(ClusterInstance cluster) throws 
Exception {
+        this.cluster = cluster;
+        try (Admin admin = cluster.admin()) {
+            cluster.createTopicWithAssignment(TOPIC, Map.of(0, 
List.of(LEADER_BROKER_ID, FOLLOWER_BROKER_ID)));
+            int leaderId = 
TestUtils.waitUntilLeaderIsKnown(cluster.brokers().values(), new 
TopicPartition(TOPIC, 0));
+
+            var topicPartition = new TopicPartition(TOPIC, 0);
+            assertEquals(LEADER_BROKER_ID, leaderId);
+
+            var fetchData = createPartitionMap(1000, List.of(topicPartition), 
Map.of(topicPartition, 0L));
+            var fetchRequest = 
FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion(), 20000, 1, 
fetchData)
+                    .setMaxBytes(1000)
+                    .rackId("")
+                    .build();
+
+            int followerPort = 
cluster.brokers().get(FOLLOWER_BROKER_ID).boundPort(cluster.clientListener());
+            try (var socket = IntegrationTestUtils.connect(followerPort)) {
+                IntegrationTestUtils.sendRequest(socket,
+                                        
Utils.toArray(fetchRequest.serializeWithHeader(
+                                            
IntegrationTestUtils.nextRequestHeader(ApiKeys.FETCH, 
ApiKeys.FETCH.latestVersion()))));
+
+                try (Producer<byte[], byte[]> producer = 
cluster.producer(Map.of())) {
+                    producer.send(new ProducerRecord<>(TOPIC, 
"key".getBytes(), "value".getBytes())).get();
+                }
+
+                FetchResponse response = IntegrationTestUtils.receive(socket, 
ApiKeys.FETCH, ApiKeys.FETCH.latestVersion());
+                assertEquals(Errors.NONE, response.error());
+                assertEquals(Map.of(Errors.NONE, 2), response.errorCounts());
+            }
+        }
+    }
+
+    @ClusterTest
+    @Timeout(15)
+    public void 
testFetchFromLeaderWhilePreferredReadReplicaIsUnavailable(ClusterInstance 
cluster) throws Exception {
+        this.cluster = cluster;
+        cluster.createTopicWithAssignment(TOPIC, Map.of(0, 
List.of(LEADER_BROKER_ID, FOLLOWER_BROKER_ID)));
+        TestUtils.waitUntilLeaderIsKnown(cluster.brokers().values(), new 
TopicPartition(TOPIC, 0));
+
+        produceMessages(10);
+        assertEquals(1, getPreferredReplica());
+
+        cluster.brokers().get(FOLLOWER_BROKER_ID).shutdown();
+        TopicPartition topicPartition = new TopicPartition(TOPIC, 0);
+
+        waitForCondition(
+                () -> {
+                    var leaderBroker = cluster.brokers().get(LEADER_BROKER_ID);
+                    var endpoints = leaderBroker.metadataCache()
+                            .getPartitionReplicaEndpoints(topicPartition, 
cluster.clientListener());
+                    return !endpoints.containsKey(FOLLOWER_BROKER_ID);
+                },
+                "follower is still reachable."
+        );
+
+        assertEquals(-1, getPreferredReplica());
+    }
+
+    @ClusterTest
+    @Timeout(60)
+    public void testFetchFromFollowerWithRoll(ClusterInstance cluster) throws 
Exception {

Review Comment:
   Yes, you are correct! testFetchFromFollowerWithRoll now creates and uses 
both followerConsumer and leaderConsumer.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to