chia7712 commented on code in PR #20199:
URL: https://github.com/apache/kafka/pull/20199#discussion_r2274299993


##########
server/src/test/java/org/apache/kafka/server/EligibleLeaderReplicasIntegrationTest.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AlterConfigOp;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.FeatureUpdate;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.admin.UpdateFeaturesOptions;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+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.server.common.EligibleLeaderReplicasVersion;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.config.ReplicationConfigs;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler;
+import org.apache.kafka.test.TestUtils;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ClusterTestDefaults(
+    brokers = 5,
+    serverProperties = {
+        @ClusterConfigProperty(key = 
ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value = "true"),
+        @ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, 
value = "true"),
+        @ClusterConfigProperty(key = 
ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, value = "4")
+    }
+)
+public class EligibleLeaderReplicasIntegrationTest {
+    private final ClusterInstance clusterInstance;
+    private String testTopicName;
+
+    EligibleLeaderReplicasIntegrationTest(ClusterInstance clusterInstance) {
+        this.clusterInstance = clusterInstance;
+    }
+
+    @ClusterTest(metadataVersion = MetadataVersion.IBP_4_0_IV1)
+    public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws 
ExecutionException, InterruptedException {
+        try (var admin = clusterInstance.admin();
+            var producer = clusterInstance.producer(Map.of(
+                ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName(),
+                ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName(),
+                ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers(),
+                ProducerConfig.ACKS_CONFIG, "1"));
+            var consumer = clusterInstance.consumer(Map.of(
+                ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers(),
+                ConsumerConfig.GROUP_ID_CONFIG, "test",
+                ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "10",
+                ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest",
+                ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName(),
+                ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName()))) {
+            testTopicName = String.format("%s-%s", 
"testHighWatermarkShouldNotAdvanceIfUnderMinIsr", "ELR-test");
+            admin.updateFeatures(
+                Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME,
+                    new 
FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), 
FeatureUpdate.UpgradeType.UPGRADE)),
+                new UpdateFeaturesOptions()
+            );
+
+            admin.createTopics(List.of(new NewTopic(testTopicName, 1, (short) 
4))).all().get();
+            clusterInstance.waitTopicCreation(testTopicName, 1);
+
+            ConfigResource configResource = new 
ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+            Collection<AlterConfigOp> ops = new ArrayList<>();
+            ops.add(new AlterConfigOp(new 
ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"), 
AlterConfigOp.OpType.SET));
+            Map<ConfigResource, Collection<AlterConfigOp>> configOps = 
Map.of(configResource, ops);
+            // alter configs on target cluster
+            admin.incrementalAlterConfigs(configOps).all().get();
+
+            TopicDescription testTopicDescription = 
admin.describeTopics(List.of(testTopicName))
+                .allTopicNames().get().get(testTopicName);
+            TopicPartitionInfo topicPartitionInfo = 
testTopicDescription.partitions().get(0);
+            List<Node> initialReplicas = topicPartitionInfo.replicas();
+            assertEquals(4, topicPartitionInfo.isr().size());
+            assertEquals(0, topicPartitionInfo.elr().size());
+            assertEquals(0, topicPartitionInfo.lastKnownElr().size());
+
+            consumer.subscribe(Set.of(testTopicName));
+            producer.send(new ProducerRecord<>(testTopicName, "0", "0")).get();
+            waitUntilOneMessageIsConsumed(consumer);
+
+            clusterInstance.shutdownBroker(initialReplicas.get(0).id());
+            clusterInstance.shutdownBroker(initialReplicas.get(1).id());
+
+            waitForIsrAndElr((isrSize, elrSize) -> isrSize == 2 && elrSize == 
1, admin);
+
+            // Now the partition is under min ISR. HWM should not advance.
+            producer.send(new ProducerRecord<>(testTopicName, "1", "1")).get();
+            TimeUnit.MILLISECONDS.sleep(100);
+            assertEquals(0, consumer.poll(Duration.ofSeconds(1L)).count());
+
+            // Restore the min ISR and the previous log should be visible.
+            clusterInstance.startBroker(initialReplicas.get(1).id());
+            clusterInstance.startBroker(initialReplicas.get(0).id());
+            waitForIsrAndElr((isrSize, elrSize) -> isrSize == 4 && elrSize == 
0, admin);
+
+            waitUntilOneMessageIsConsumed(consumer);
+        }
+    }
+
+    void waitUntilOneMessageIsConsumed(Consumer<?, ?> consumer) throws 
InterruptedException {
+        TestUtils.waitForCondition(
+            () -> {
+                try {
+                    ConsumerRecords record = 
consumer.poll(Duration.ofMillis(100L));

Review Comment:
   ```java
   return consumer.poll(Duration.ofMillis(100L)).count() >= 1;
   ```



##########
server/src/test/java/org/apache/kafka/server/EligibleLeaderReplicasIntegrationTest.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AlterConfigOp;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.FeatureUpdate;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.admin.UpdateFeaturesOptions;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+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.server.common.EligibleLeaderReplicasVersion;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.config.ReplicationConfigs;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler;
+import org.apache.kafka.test.TestUtils;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ClusterTestDefaults(
+    brokers = 5,
+    serverProperties = {
+        @ClusterConfigProperty(key = 
ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value = "true"),
+        @ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, 
value = "true"),
+        @ClusterConfigProperty(key = 
ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, value = "4")
+    }
+)
+public class EligibleLeaderReplicasIntegrationTest {
+    private final ClusterInstance clusterInstance;
+    private String testTopicName;
+
+    EligibleLeaderReplicasIntegrationTest(ClusterInstance clusterInstance) {
+        this.clusterInstance = clusterInstance;
+    }
+
+    @ClusterTest(metadataVersion = MetadataVersion.IBP_4_0_IV1)

Review Comment:
   Could you please add `Type=Raft-Isolated`? It does not need to run all types.



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