CalvinConfluent commented on code in PR #18277:
URL: https://github.com/apache/kafka/pull/18277#discussion_r1938076119


##########
core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java:
##########
@@ -0,0 +1,450 @@
+/*
+ * 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 kafka.server.integration;
+import kafka.integration.KafkaServerTestHarness;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+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.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.common.EligibleLeaderReplicasVersion;
+import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+import scala.collection.mutable.HashMap;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class EligibleLeaderReplicasIntegrationTest extends 
KafkaServerTestHarness implements Logging {
+    private String bootstrapServer;
+    private String testTopicName;
+    private Admin adminClient;
+    @Override
+    public Seq<KafkaConfig> generateConfigs() {
+        List<Properties> brokerConfigs = new ArrayList<>();
+        
brokerConfigs.addAll(scala.collection.JavaConverters.seqAsJavaList(TestUtils.createBrokerConfigs(
+            5,
+            true,
+            true,
+            scala.Option.<SecurityProtocol>empty(),
+            scala.Option.<File>empty(),
+            scala.Option.<Properties>empty(),
+            true,
+            false,
+            false,
+            false,
+            new HashMap<>(),
+            1,
+            false,
+            1,
+            (short) 4,
+            0,
+            false
+        )));
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, 
bootstrapServer);
+        adminClient = Admin.create(props);
+        adminClient.updateFeatures(
+            Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME,
+                new 
FeatureUpdate(EligibleLeaderReplicasVersion.ELRV_1.featureLevel(), 
FeatureUpdate.UpgradeType.UPGRADE)),
+            new UpdateFeaturesOptions()
+        );
+        testTopicName = String.format("%s-%s", 
info.getTestMethod().get().getName(), "ELR-test");
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (adminClient != null) adminClient.close();
+        super.tearDown();
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"kraft"})
+    public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr(String quorum) 
throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 
4))).all().get();
+        TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
+
+        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 = 
Collections.singletonMap(configResource, ops);
+        // alter configs on target cluster
+        adminClient.incrementalAlterConfigs(configOps).all().get();
+        Producer producer = null;
+        Consumer consumer = null;
+        try {
+            // check which partition is on broker 0 which we'll kill

Review Comment:
   Looks like this comment is not useful. Removed.



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