Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion kafka/tools/assigner/__main__.py
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ def main():
tools_path = get_tools_path(args.tools_path)
check_java_home()

cluster = Cluster.create_from_zookeeper(args.zookeeper)
cluster = Cluster.create_from_zookeeper(args.zookeeper, args.use_active_brokers)
run_plugins_at_step(plugins, 'set_cluster', cluster)

# If the module needs the partition sizes, call a size module to get the information
Expand Down
1 change: 1 addition & 0 deletions kafka/tools/assigner/arguments.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ def set_up_arguments(action_map, sizer_map, plugins):
aparser.add_argument('--ple-size', help="Max number of partitions in a single preferred leader election", required=False, default=2000, type=int)
aparser.add_argument('--ple-wait', help="Time in seconds to wait between preferred leader elections", required=False, default=120, type=int)
aparser.add_argument('--tools-path', help="Path to Kafka admin utilities, overriding PATH env var", required=False)
aparser.add_argument('--use-active-brokers', help="Use only active brokers as ZK sets in /brokers/ids/", action='store_true')

# Call action module arg setup
subparsers = aparser.add_subparsers(help='Select manipulation module to use')
Expand Down
24 changes: 16 additions & 8 deletions kafka/tools/assigner/models/cluster.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,15 +33,23 @@ def add_brokers_from_zk(cluster, zk):
raise ZookeeperException("The cluster specified does not have any brokers")


def add_topic_with_replicas(cluster, topic, topic_data):
def add_replica_to_broker_list(cluster, i, replica, newtopic, partition, use_active_brokers):
if replica not in cluster.brokers and not use_active_brokers:
# Hit a replica that's not in the ID list (which means it's dead)
# We'll add it, but trying to get sizes will fail as we don't have a hostname

# Only add these brokers that are not present in ZK if use_active_brokers is False
cluster.add_broker(Broker(replica, None))

if replica in cluster.brokers:
newtopic.partitions[int(partition)].add_replica(cluster.brokers[replica], i)


def add_topic_with_replicas(cluster, topic, topic_data, use_active_brokers):
newtopic = Topic(topic, len(topic_data['partitions']))
for partition in topic_data['partitions']:
for i, replica in enumerate(topic_data['partitions'][partition]):
if replica not in cluster.brokers:
# Hit a replica that's not in the ID list (which means it's dead)
# We'll add it, but trying to get sizes will fail as we don't have a hostname
cluster.add_broker(Broker(replica, None))
newtopic.partitions[int(partition)].add_replica(cluster.brokers[replica], i)
add_replica_to_broker_list(cluster, i, replica, newtopic, partition, use_active_brokers)
cluster.add_topic(newtopic)


Expand All @@ -53,7 +61,7 @@ def __init__(self):
self.topics = {}

@classmethod
def create_from_zookeeper(cls, zkconnect):
def create_from_zookeeper(cls, zkconnect, use_active_brokers=False):
log.info("Connecting to zookeeper {0}".format(zkconnect))
try:
zk = KazooClient(zkconnect)
Expand All @@ -69,7 +77,7 @@ def create_from_zookeeper(cls, zkconnect):
log.info("Getting partition list from Zookeeper")
for topic in zk.get_children("/brokers/topics"):
zdata, zstat = zk.get("/brokers/topics/{0}".format(topic))
add_topic_with_replicas(cluster, topic, json.loads(zdata))
add_topic_with_replicas(cluster, topic, json.loads(zdata), use_active_brokers)

if cluster.num_topics() == 0:
raise ZookeeperException("The cluster specified does not have any topics")
Expand Down
16 changes: 15 additions & 1 deletion tests/tools/assigner/models/test_cluster.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@

from kafka.tools.assigner.models.broker import Broker
from kafka.tools.assigner.models.topic import Topic
from kafka.tools.assigner.models.cluster import Cluster
from kafka.tools.assigner.models.cluster import Cluster, add_topic_with_replicas


class SimpleClusterTests(unittest.TestCase):
Expand Down Expand Up @@ -70,3 +70,17 @@ def test_cluster_log_info(self):
self.cluster.log_broker_summary()
l.check(('kafka-assigner', 'INFO', 'Broker 1: partitions=0/0 (0.00%), size=0'),
('kafka-assigner', 'INFO', 'Broker 2: partitions=0/0 (0.00%), size=0'))

def test_add_topic_with_replicas_active_brokers(self):
self.add_brokers(2)
topic = "test_topic"
topic_data = {'partitions': {0: [1111, 2222, 1]}} # Only 1 is an active broker
add_topic_with_replicas(self.cluster, topic, topic_data, use_active_brokers=True)
assert len(self.cluster.brokers) == 2

def test_add_topic_with_replicas(self):
self.add_brokers(2)
topic = "test_topic"
topic_data = {'partitions': {0: [1111, 2222, 1]}} # Add all brokers
add_topic_with_replicas(self.cluster, topic, topic_data, use_active_brokers=False)
assert len(self.cluster.brokers) == 4
3 changes: 2 additions & 1 deletion tests/tools/assigner/test_main.py
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,8 @@ def test_main(self, mock_plugins, mock_sizes):
ple_size=2,
ple_wait=120,
sizer='ssh',
leadership=True)
leadership=True,
use_active_brokers=False)
assert main() == 0

def test_get_plugins(self):
Expand Down