You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Ivan Yurchenko (Jira)" <ji...@apache.org> on 2020/01/28 16:08:00 UTC

[jira] [Created] (KAFKA-9478) Controller may stop react on partition reassignment command in ZooKeeper

Ivan Yurchenko created KAFKA-9478:
-------------------------------------

             Summary: Controller may stop react on partition reassignment command in ZooKeeper
                 Key: KAFKA-9478
                 URL: https://issues.apache.org/jira/browse/KAFKA-9478
             Project: Kafka
          Issue Type: Bug
          Components: controller, core
    Affects Versions: 2.4.0, 2.4.1
            Reporter: Ivan Yurchenko
            Assignee: Ivan Yurchenko


Seemingly after [bdf2446ccce592f3c000290f11de88520327aa19|https://github.com/apache/kafka/commit/bdf2446ccce592f3c000290f11de88520327aa19], the controller may stop watching {{/admin/reassign_partitions}} node in ZooKeeper and consequently accept partition reassignment commands via ZooKeeper.

I'm not 100% sure that bdf2446ccce592f3c000290f11de88520327aa19 causes this, but it doesn't reproduce on [3fe6b5e951db8f7184a4098f8ad8a1afb2b2c1a0|https://github.com/apache/kafka/commit/3fe6b5e951db8f7184a4098f8ad8a1afb2b2c1a0] - the one right before it.

Also, reproduces on the trunk HEAD [a87decb9e4df5bfa092c26ae4346f65c426f1321|https://github.com/apache/kafka/commit/a87decb9e4df5bfa092c26ae4346f65c426f1321].
h1. How to reproduce

1. Run ZooKeeper and two Kafka brokers.

2. Create a topic with 100 partitions and place them on Broker 0:
{code:bash}
distro/bin/kafka-topics.sh --bootstrap-server localhost:9092,localhost:9093 --create \
    --topic foo \
    --replica-assignment $(for i in {0..99}; do echo -n "0,"; done | sed 's/.$$//')
{code}
3. Add some data:
{code:bash}
seq 1 1000000 | bin/kafka-console-producer.sh --broker-list localhost:9092,localhost:9093 --topic foo
{code}
4. Create the partition reassignment node {{/admin/reassign_partitions}} in Zoo and shortly after that update the data in the node (even the same value will do). I made a simple Python script for this:
{code:python}
import time
import json
from kazoo.client import KazooClient

zk = KazooClient(hosts='127.0.0.1:2181')
zk.start()

reassign = {
	"version": 1,
	"partitions":[]
}
for p in range(100):
	reassign["partitions"].append({"topic": "foo", "partition": p, "replicas": [1]})

zk.create("/admin/reassign_partitions", json.dumps(reassign).encode())

time.sleep(0.05)

zk.set("/admin/reassign_partitions", json.dumps(reassign).encode())
{code}
4. Observe that the controller doesn't react on further updates to {{/admin/reassign_partitions}} and doesn't delete the node.

Also, it can be confirmed with
{code:bash}
echo wchc | nc 127.0.0.1 2181
{code}
that there is no watch on the node in ZooKeeper (for this, you should run ZooKeeper with {{4lw.commands.whitelist=*}}).

Since it's about timing, it might not work on first attempt, so you might need to do 4 a couple of times. However, the reproducibility rate is pretty high.

The data in the topic and the big amount of partitions are not needed per se, only to make the timing more favourable.

Controller re-election will solve the issue, but a new controller can be put in this state the same way.
h1. Proposed solution

TBD, suggestions are welcome.

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)