You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "soarez (via GitHub)" <gi...@apache.org> on 2023/05/18 08:04:29 UTC

[GitHub] [kafka] soarez commented on a diff in pull request #13558: KAFKA-14845: Fix broker registration with Zookeeper when the previous ephemeral znode was not properly recorded by the broker

soarez commented on code in PR #13558:
URL: https://github.com/apache/kafka/pull/13558#discussion_r1197495849


##########
core/src/test/resources/log4j.properties:
##########
@@ -20,7 +20,8 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n
 
 log4j.logger.kafka=WARN
 log4j.logger.org.apache.kafka=WARN
-
+log4j.logger.kafka.zk=INFO
+log4j.logger.org.apache.zookeeper.server=INFO

Review Comment:
   Do we really want to enable this for all tests? Should we instead leave a comment on the new test that suggests enabling these for troubleshooting as needed?



##########
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:
   It's a bit odd to keep an extra thread just waiting for a signal to shut down ZK. Could we not have `start()` and `stop()` methods instead?



##########
core/src/test/java/kafka/zk/ReceiptEvent.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.zookeeper.server.MutedServerCxn;
+import org.apache.zookeeper.server.Request;
+import org.apache.zookeeper.server.ServerCnxn;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Event corresponding to the invocation of the Zookeeper request processor for a request.
+ */
+public class ReceiptEvent {
+    private final int opCode;
+    private final Lock lock = new ReentrantLock();
+    private final Condition received = lock.newCondition();
+    private final Condition processed = lock.newCondition();
+    private State state = State.notReceived;
+    private boolean sendResponse;
+
+    public ReceiptEvent(int opCode, boolean sendResponse) {
+        this.opCode = opCode;
+        this.sendResponse = sendResponse;
+    }
+
+    public boolean matches(Request request) {
+        return request.type == opCode;
+    }
+
+    public void serverReceived() {
+        lock.lock();
+        try {
+            state = Collections.max(Arrays.asList(state, State.received));
+            received.signalAll();

Review Comment:
   This condition doesn't seem to be used anywhere?



##########
core/src/test/java/kafka/zk/ReceiptEvent.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.zookeeper.server.MutedServerCxn;
+import org.apache.zookeeper.server.Request;
+import org.apache.zookeeper.server.ServerCnxn;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Event corresponding to the invocation of the Zookeeper request processor for a request.
+ */
+public class ReceiptEvent {
+    private final int opCode;
+    private final Lock lock = new ReentrantLock();
+    private final Condition received = lock.newCondition();
+    private final Condition processed = lock.newCondition();

Review Comment:
   Could this maybe be lighter on locking by using with a `AtomicReference<State>.compareAndSet()` on `serverProcessed()` and conditionally completing a `CompletableFuture<Void> processed` for `awaitProcessed()` to `.get()` on?



##########
checkstyle/suppressions.xml:
##########
@@ -41,6 +41,8 @@
     <suppress checks="NPathComplexity" files="(ClusterTestExtensions|KafkaApisBuilder).java"/>
     <suppress checks="MethodLength"
               files="(KafkaClusterTestKit).java"/>
+    <suppress checks="ImportControl"
+              files="(ZkBrokerRegistrationTest|DelegatingRequest|InstrumentedRequestProcessor|InstrumentedZooKeeperServer|ReceiptEvent|MutedServerCxn).java"/>

Review Comment:
   Why add this exception instead of sorting the import statements as per checkstyle rules?



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

Review Comment:
   This doesn't seem to be used, can be local.



##########
core/src/main/scala/kafka/zk/KafkaZkClient.scala:
##########
@@ -2113,7 +2113,12 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
   }
 
   private class CheckedEphemeral(path: String, data: Array[Byte]) extends Logging {
+    private var attempt = 0
+    private val maxAttempt = 5
+    private val backoffMs = 1000

Review Comment:
   Should this be a function of the `"zookeeper.session.timeout.ms"` config property?



##########
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();
+                    }

Review Comment:
   Can't this be part of `InstrumentedZooKeeperServer`, since it's a new class anyway? 



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

Review Comment:
   Since we're finding a free port and starting zookeeper and everything, shouldn't we tag as an integration test using `@Tag("integration")`?



-- 
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-unsubscribe@kafka.apache.org

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