soarez commented on code in PR #13558: URL: https://github.com/apache/kafka/pull/13558#discussion_r1198793889
########## core/src/test/java/kafka/zk/ZkBrokerRegistrationTest.java: ########## @@ -0,0 +1,298 @@ +/* + * 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.zk; + +import io.netty.channel.group.ChannelGroup; +import io.netty.util.concurrent.EventExecutor; +import kafka.cluster.Broker; +import kafka.server.KafkaConfig; +import kafka.zookeeper.ZooKeeperClient; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.Time; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.server.PrepRequestProcessor; +import org.apache.zookeeper.server.RequestProcessor; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.SyncRequestProcessor; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Field; +import java.net.ServerSocket; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; + +import static java.util.Arrays.asList; +import static org.apache.kafka.common.security.auth.SecurityProtocol.PLAINTEXT; +import static org.apache.zookeeper.client.ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET; + +/** + * This test simulates the loss of a session ID with Zookeeper (for instance due to network partition) + * while a broker is in the process of creating its ephemeral znode under /brokers/ids/. This can + * result in broker registration failing with due to a NODEEXISTS error which is not handled by the + * fix KAFKA-6584 because in this case, the ID of the session of the conflicting ephemeral znode is + * not known by the broker. See KAFKA-14845 for an example of timeline of events requires to reproduce + * the use case. + */ +public class ZkBrokerRegistrationTest { + private static final Logger log = LoggerFactory.getLogger(ZkBrokerRegistrationTest.class); + + private KafkaConfig kafkaConfig; + private BrokerInfo brokerInfo; + private int zkPort; + + private class SandboxedZookeeper extends Thread { + private final CountDownLatch zookeeperStopLatch; + private final CountDownLatch zookeeperStartLatch = new CountDownLatch(1); + private final ZkTestContext spec; + private InstrumentedRequestProcessor processor; + + SandboxedZookeeper(CountDownLatch zookeeperStopLatch, ZkTestContext spec) { + this.zookeeperStopLatch = zookeeperStopLatch; + this.spec = spec; + } + + public void run() { + ServerCnxnFactory cnxnFactory = null; + + try { + Path dataDir = Files.createTempDirectory("zk"); + + Properties zkProperties = new Properties(); + zkProperties.put("dataDir", dataDir.toFile().getPath()); + zkProperties.put("clientPort", String.valueOf(zkPort)); + zkProperties.put("serverCnxnFactory", "org.apache.zookeeper.server.NettyServerCnxnFactory"); + + QuorumPeerConfig config = new QuorumPeerConfig(); + config.parseProperties(zkProperties); + FileTxnSnapLog txnLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir()); + + ZooKeeperServer zookeeper = new InstrumentedZooKeeperServer( + null, + txnLog, + config.getTickTime(), + config.getMinSessionTimeout(), + config.getMaxSessionTimeout(), + config.getClientPortListenBacklog(), + null, + config.getInitialConfig(), + spec) { + + @Override + protected void setupRequestProcessors() { + processor = new InstrumentedRequestProcessor(this, spec); + RequestProcessor syncProcessor = new SyncRequestProcessor(this, processor); + ((SyncRequestProcessor) syncProcessor).start(); + firstProcessor = new PrepRequestProcessor(this, syncProcessor); + ((PrepRequestProcessor) firstProcessor).start(); + } + }; + + cnxnFactory = ServerCnxnFactory.createFactory(); + cnxnFactory.configure( + config.getClientPortAddress(), + config.getMaxClientCnxns(), + config.getClientPortListenBacklog(), + false); + cnxnFactory.startup(zookeeper); + + zookeeperStartLatch.countDown(); + zookeeperStopLatch.await(); Review Comment: Indeed, I didn't mean to suggest `Thread#stop()`. Currently my understanding is that `SandoxedZookeeper` is a thread that when run, starts ZK (asynchronously) and then that same thread just actively waits for a signal before it shutdowns ZK. What I'm proposing is that `SandoxedZookeeper` not be a `Thread`, and provide a `startZk()` (or just `start()` method that does what is currently being done in `Thread#run()` to bring up ZK, and also provide a `stop()` method to do what is currently being done after the signal is received. i.e. instead of propagating a signal, we could just invoke `stop()`. I'm hoping this way we can avoid having an extra Thread, just actively waiting to shutdown ZK. Would this make sense? Or am I missing something maybe? -- 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