mumrah commented on a change in pull request #10220:
URL: https://github.com/apache/kafka/pull/10220#discussion_r597214017



##########
File path: core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
##########
@@ -0,0 +1,499 @@
+/*
+ * 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.testkit;
+
+import kafka.raft.KafkaRaftManager;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaConfig$;
+import kafka.server.Server;
+import kafka.server.BrokerServer;
+import kafka.server.ControllerServer;
+import kafka.server.MetaProperties;
+import kafka.tools.StorageTool;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.utils.ThreadUtils;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.controller.Controller;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.raft.RaftConfig;
+import org.apache.kafka.raft.metadata.MetaLogRaftShim;
+import org.apache.kafka.raft.metadata.MetadataRecordSerde;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+
+@SuppressWarnings("deprecation") // Needed for Scala 2.12 compatibility
+public class KafkaClusterTestKit implements AutoCloseable {
+    private final static Logger log = 
LoggerFactory.getLogger(KafkaClusterTestKit.class);
+
+    /**
+     * This class manages a future which is completed with the proper value for
+     * controller.quorum.voters once the randomly assigned ports for all the 
controllers are
+     * known.
+     */
+    private static class ControllerQuorumVotersFutureManager implements 
AutoCloseable {
+        private final int expectedControllers;
+        private final CompletableFuture<List<String>> future = new 
CompletableFuture<>();
+        private final Map<Integer, Integer> controllerPorts = new TreeMap<>();
+
+        ControllerQuorumVotersFutureManager(int expectedControllers) {
+            this.expectedControllers = expectedControllers;
+        }
+
+        synchronized void registerPort(int nodeId, int port) {
+            controllerPorts.put(nodeId, port);
+            if (controllerPorts.size() >= expectedControllers) {
+                future.complete(controllerPorts.entrySet().stream().
+                    map(e -> String.format("%d@localhost:%d", e.getKey(), 
e.getValue())).
+                    collect(Collectors.toList()));
+            }
+        }
+
+        void fail(Throwable e) {
+            future.completeExceptionally(e);
+        }
+
+        @Override
+        public void close() {
+            future.cancel(true);
+        }
+    }
+
+    public static class Builder {
+        private TestKitNodes nodes;
+        private Map<String, String> configProps = new HashMap<>();
+
+        public Builder(TestKitNodes nodes) {
+            this.nodes = nodes;
+        }
+
+        public Builder setConfigProp(String key, String value) {
+            this.configProps.put(key, value);
+            return this;
+        }
+
+        public KafkaClusterTestKit build() throws Exception {
+            Map<Integer, ControllerServer> controllers = new HashMap<>();
+            Map<Integer, BrokerServer> kip500Brokers = new HashMap<>();
+            Map<Integer, KafkaRaftManager> raftManagers = new HashMap<>();
+            String dummyQuorumVotersString = 
nodes.controllerNodes().keySet().stream().
+                map(controllerNode -> String.format("%d@0.0.0.0:0", 
controllerNode)).
+                collect(Collectors.joining(","));
+            /*
+              Number of threads = Total number of brokers + Total number of 
controllers + Total number of Raft Managers
+                                = Total number of brokers + Total number of 
controllers * 2
+                                  (Raft Manager per broker/controller)
+             */
+            int numOfExecutorThreads = (nodes.brokerNodes().size() + 
nodes.controllerNodes().size()) * 2;
+            ExecutorService executorService = null;
+            ControllerQuorumVotersFutureManager connectFutureManager =
+                new 
ControllerQuorumVotersFutureManager(nodes.controllerNodes().size());
+            File baseDirectory = null;
+
+            try {
+                baseDirectory = TestUtils.tempDirectory();
+                nodes = 
nodes.copyWithAbsolutePaths(baseDirectory.getAbsolutePath());
+                executorService = 
Executors.newFixedThreadPool(numOfExecutorThreads,
+                    ThreadUtils.createThreadFactory("KafkaClusterTestKit%d", 
false));
+                Time time = Time.SYSTEM;
+                for (ControllerNode node : nodes.controllerNodes().values()) {

Review comment:
       I took a stab at this and it didn't really end up much cleaner




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

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


Reply via email to