From 49e5db8b838148ee0e5e5ac90f9b87dbc66e709c Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 3 Jan 2018 15:21:26 -0800 Subject: [PATCH 1/9] add admin_client api --- kafka/protocol/admin.py | 2 +- run_itest.sh | 6 ++++++ test/fixtures.py | 16 +--------------- test/testutil.py | 2 ++ tox.ini | 1 + 5 files changed, 11 insertions(+), 16 deletions(-) diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index c5142b3ec..17fa14a51 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -88,7 +88,7 @@ class CreateTopicsRequest_v1(Request): CreateTopicsRequest = [CreateTopicsRequest_v0, CreateTopicsRequest_v1] -CreateTopicsResponse = [CreateTopicsResponse_v0, CreateTopicsRequest_v1] +CreateTopicsResponse = [CreateTopicsResponse_v0, CreateTopicsResponse_v1] class DeleteTopicsResponse_v0(Response): diff --git a/run_itest.sh b/run_itest.sh index a55802b52..fcc78613f 100644 --- a/run_itest.sh +++ b/run_itest.sh @@ -12,3 +12,9 @@ export KAFKA_VERSION='0.10.0.0' tox -e py27 tox -e py35 tox -e pypy + +export KAFKA_VERSION='0.10.1.1' +./build_integration.sh +tox -e py27 +tox -e py35 +tox -e pypy diff --git a/test/fixtures.py b/test/fixtures.py index bfb01b8e5..839bae6d9 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -191,22 +191,8 @@ def instance(cls, broker_id, zk_host, zk_port, zk_chroot=None, (host, port) = (parse.hostname, parse.port) fixture = ExternalService(host, port) else: - # force IPv6 here because of a confusing point: - # - # - if the string "localhost" is passed, Kafka will *only* bind to the IPv4 address of localhost - # (127.0.0.1); however, kafka-python will attempt to connect on ::1 and fail - # - # - if the address literal 127.0.0.1 is passed, the metadata request during bootstrap will return - # the name "localhost" and we'll go back to the first case. This is odd! - # - # Ideally, Kafka would bind to all loopback addresses when we tell it to listen on "localhost" the - # way it makes an IPv6 socket bound to both 0.0.0.0/0 and ::/0 when we tell it to bind to "" (that is - # to say, when we make a listener of PLAINTEXT://:port. - # - # Note that even though we specify the bind host in bracket notation, Kafka responds to the bootstrap - # metadata request without square brackets later. if host is None: - host = "[::1]" + host = "localhost" fixture = KafkaFixture(host, port, broker_id, zk_host, zk_port, zk_chroot, transport=transport, diff --git a/test/testutil.py b/test/testutil.py index c247e6ad7..3a9773a8e 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -12,6 +12,7 @@ from . import unittest from kafka import SimpleClient +from kafka.client_async import KafkaClient from kafka.structs import OffsetRequestPayload __all__ = [ @@ -97,6 +98,7 @@ def setUp(self): if self.create_client: self.client = SimpleClient('%s:%d' % (self.server.host, self.server.port)) + self.client_async = KafkaClient(bootstrap_servers='%s:%d' % (self.server.host, self.server.port)) self.client.ensure_topic_exists(self.topic) diff --git a/tox.ini b/tox.ini index 03a6893ad..e6ee86ae2 100644 --- a/tox.ini +++ b/tox.ini @@ -9,6 +9,7 @@ log_format = %(created)f %(filename)-23s %(threadName)s %(message)s [testenv] deps = + pylint==1.7.1 pytest pytest-cov pytest-catchlog From 607560bbf3ff5950f6fda9186d8b7bd081b055f9 Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 3 Jan 2018 15:22:52 -0800 Subject: [PATCH 2/9] add admin_client api --- kafka/admin_client.py | 136 ++++++++++++++++++++++++++ test/test_admin_client.py | 78 +++++++++++++++ test/test_admin_client_integration.py | 46 +++++++++ 3 files changed, 260 insertions(+) create mode 100644 kafka/admin_client.py create mode 100644 test/test_admin_client.py create mode 100644 test/test_admin_client_integration.py diff --git a/kafka/admin_client.py b/kafka/admin_client.py new file mode 100644 index 000000000..69f3d3903 --- /dev/null +++ b/kafka/admin_client.py @@ -0,0 +1,136 @@ +import time +from .errors import NodeNotReadyError +from .protocol.admin import CreateTopicsRequest, DeleteTopicsRequest +from .protocol.metadata import MetadataRequest + +def convert_new_topic_request_format(new_topic): + return ( + new_topic.name, + new_topic.num_partitions, + new_topic.replication_factor, + [ + (partition_id,replicas) + for partition_id, replicas in new_topic.replica_assignments.items() + ], + [ + (config_key, config_value) + for config_key, config_value in new_topic.configs.items() + ], + ) + +class NewTopic(object): + """ A class for new topic creation + + Arguments: + name (string): name of the topic + num_partitions (int): number of partitions + or -1 if replica_assignment has been specified + replication_factor (int): replication factor or -1 if + replica assignment is specified + replica_assignment (dict of int: [int]): A mapping containing + partition id and replicas to assign to it. + topic_configs (dict of str: str): A mapping of config key + and value for the topic. + """ + + def __init__( + self, + name, + num_partitions, + replication_factor, + replica_assignments=None, + configs=None, + ): + self.name = name + self.configs = configs or {} + self.num_partitions = num_partitions + self.replication_factor = replication_factor + self.replica_assignments = replica_assignments or {} + + def __str__(self): + return ":{}, :{}, :{}" \ + ":{}, :{}".format( + self.name, + self.num_partitions, + self.replication_factor, + self.replica_assignments, + self.configs, + ) + +class AdminClient(object): + """ + An api to send CreateTopic requests + + """ + def __init__(self, client): + self.client = client + self.metadata_request = MetadataRequest[1]([]) + self.topic_request = CreateTopicsRequest[0] + self.delete_topics_request = DeleteTopicsRequest[0] + + def _send_controller_request(self): + response = self._send( + self.client.least_loaded_node(), + self.metadata_request, + ) + return response[0].controller_id + + def _send(self, node, request): + future = self.client.send(node, request) + return self.client.poll(future=future) + + def _send_request(self, request): + controller_id = self._send_controller_request() + if not self.client.ready(controller_id): + raise NodeNotReadyError(controller_id) + else: + return self._send(controller_id, request) + + + def create_topics( + self, + topics, + timeout, + ): + """ Create topics on the cluster + + Arguments: + new_topics (list of NewTopic): A list containing new + topics to be created + validate_only (bool): True if we just want to validate the request + timeout (int): timeout in seconds + max_retry (int): num of times we want to retry to send a create + topic request when the controller in not available + + Returns: + CreateTopicResponse: response from the broker + + Raises: + NodeNotReadyError: if controller is not ready + """ + request = self.topic_request( + create_topic_requests=[ + convert_new_topic_request_format(topic) + for topic in topics + ], + timeout=timeout, + ) + return self._send_request(request) + + def delete_topics(self, topics, timeout): + """ Deletes topics on the cluster + + Arguments: + topics (list of topic names): Topics to delete + timeout (int): The requested timeout for this operation + Raises: + NodeNotReadyError: if retry exceeds max_retry + """ + + request = self.delete_topics_request( + topics=topics, + timeout=timeout, + ) + return self._send_request(request) + + diff --git a/test/test_admin_client.py b/test/test_admin_client.py new file mode 100644 index 000000000..4f5d89544 --- /dev/null +++ b/test/test_admin_client.py @@ -0,0 +1,78 @@ +import mock +import pytest +from kafka.client_async import KafkaClient +from kafka.errors import BrokerNotAvailableError +from kafka.protocol.metadata import MetadataResponse +from kafka.protocol.admin import CreateTopicsResponse +from kafka.admin_client import AdminClient +from kafka.admin_client import NewTopic +from kafka.structs import BrokerMetadata +from kafka.future import Future + +@pytest.fixture +def bootstrap_brokers(): + return 'fake-broker:9092' + +@pytest.fixture +def controller_id(): + return 100 + +@pytest.fixture +def mock_least_loaded_node(): + return 2 + +@pytest.fixture +def metadata_response(controller_id): + return [MetadataResponse[1]( + [(1,'host',80,'rack')], controller_id, + [(37,'topic',False,[(7,1,2,[1,2,3],[1,2,3])])] + )] + +@pytest.fixture +def mock_new_topics(): + return [NewTopic('topic',1,1)] + +@pytest.fixture +def topic_response(): + return CreateTopicsResponse[1]([( + 'topic',7,'timeout_exception' + )]) + + +class TestTopicAdmin(): + + def test_send_controller_request( + self, + mock_least_loaded_node, + controller_id, + bootstrap_brokers, + metadata_response + ): + mock_kafka_client = mock.Mock() + mock_kafka_client.poll.return_value = metadata_response + mock_kafka_client.least_loaded_node.return_value = \ + mock_least_loaded_node + mock_kafka_client.send.return_value = Future() + mock_kafka_client.connected.return_value = True + admin = AdminClient(mock_kafka_client) + assert admin._send_controller_request() == controller_id + + def test_create_topics( + self, + mock_new_topics, + mock_least_loaded_node, + bootstrap_brokers, + topic_response, + metadata_response, + ): + mock_kafka_client = mock.Mock() + mock_kafka_client.poll = \ + mock.Mock(side_effect=[metadata_response, topic_response]) + mock_kafka_client.ready.return_value = True + mock_kafka_client.least_loaded_node.return_value = \ + mock_least_loaded_node + mock_kafka_client.send.return_value = Future() + admin = AdminClient(mock_kafka_client) + response = admin.create_topics(mock_new_topics, False) + assert response == topic_response + diff --git a/test/test_admin_client_integration.py b/test/test_admin_client_integration.py new file mode 100644 index 000000000..3a1a748a5 --- /dev/null +++ b/test/test_admin_client_integration.py @@ -0,0 +1,46 @@ +import os +import time +import unittest +from kafka.admin_client import AdminClient, NewTopic +from kafka.protocol.metadata import MetadataRequest +from test.fixtures import ZookeeperFixture, KafkaFixture +from test.testutil import KafkaIntegrationTestCase, kafka_versions + +class TestKafkaAdminClientIntegration(KafkaIntegrationTestCase): + + @classmethod + def setUpClass(cls): + if not os.environ.get('KAFKA_VERSION'): + return + + cls.zk = ZookeeperFixture.instance() + cls.server = KafkaFixture.instance(0, cls.zk.host, cls.zk.port) + + @classmethod + def tearDownClass(cls): + if not os.environ.get('KAFKA_VERSION'): + return + + cls.server.close() + cls.zk.close() + + @kafka_versions('>=0.10.1') + def test_create_delete_topics(self): + admin = AdminClient(self.client_async) + topic = NewTopic( + name='topic', + num_partitions=1, + replication_factor=1, + ) + metadata_request = MetadataRequest[1]() + response = admin.create_topics(topics=[topic], timeout=1) + self.assertTrue( + response[0].topic_error_codes[0][1] == 0 or + response[0].topic_error_codes[0][1] == 7 + ) + time.sleep(1) # allows the topic to be created + delete_response = admin.delete_topics(['topic'], timeout=1) + self.assertTrue( + response[0].topic_error_codes[0][1] == 0 or + response[0].topic_error_codes[0][1] == 7 + ) From 8667c3672e10e09177b295a62446bdf00aa51553 Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 3 Jan 2018 16:14:04 -0800 Subject: [PATCH 3/9] remove pylint version pin --- tox.ini | 1 - 1 file changed, 1 deletion(-) diff --git a/tox.ini b/tox.ini index e6ee86ae2..03a6893ad 100644 --- a/tox.ini +++ b/tox.ini @@ -9,7 +9,6 @@ log_format = %(created)f %(filename)-23s %(threadName)s %(message)s [testenv] deps = - pylint==1.7.1 pytest pytest-cov pytest-catchlog From b4aab4e9d9363fb44666099ca8460c04ff9a38b3 Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 3 Jan 2018 16:30:51 -0800 Subject: [PATCH 4/9] turns theres an issue with pylint even on travis pinning pylint again --- tox.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/tox.ini b/tox.ini index 03a6893ad..e6ee86ae2 100644 --- a/tox.ini +++ b/tox.ini @@ -9,6 +9,7 @@ log_format = %(created)f %(filename)-23s %(threadName)s %(message)s [testenv] deps = + pylint==1.7.1 pytest pytest-cov pytest-catchlog From aea7a86e15079d70c503422bed75abe745e10c1d Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 3 Jan 2018 17:07:06 -0800 Subject: [PATCH 5/9] lets see if this passes travis --- tox.ini | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/tox.ini b/tox.ini index e6ee86ae2..d07670207 100644 --- a/tox.ini +++ b/tox.ini @@ -9,16 +9,16 @@ log_format = %(created)f %(filename)-23s %(threadName)s %(message)s [testenv] deps = - pylint==1.7.1 pytest pytest-cov pytest-catchlog - py{27,34,35,py}: pytest-pylint + py{27,34,35,36,py}: pylint==1.8.0 + py{27,34,35,36,py}: pytest-pylint pytest-sugar pytest-mock mock python-snappy - lz4 + lz4==0.11.1 xxhash py26: unittest2 commands = @@ -31,11 +31,6 @@ passenv = KAFKA_VERSION # pylint doesn't support python2.6 commands = py.test {posargs:--cov=kafka --cov-config=.covrc} -[testenv:py36] -# pylint doesn't support python3.6 yet -# https://github.com/PyCQA/pylint/issues/1072 -commands = py.test {posargs:--cov=kafka --cov-config=.covrc} - [testenv:pypy] # pylint is super slow on pypy... commands = py.test {posargs:--cov=kafka --cov-config=.covrc} From bc935a80a349f78637d6b293dd3320d04ea64b04 Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Thu, 4 Jan 2018 17:48:12 -0800 Subject: [PATCH 6/9] fix docstrings --- kafka/admin_client.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/kafka/admin_client.py b/kafka/admin_client.py index 69f3d3903..67152f9c2 100644 --- a/kafka/admin_client.py +++ b/kafka/admin_client.py @@ -95,12 +95,9 @@ def create_topics( """ Create topics on the cluster Arguments: - new_topics (list of NewTopic): A list containing new + topics (list of NewTopic): A list containing new topics to be created - validate_only (bool): True if we just want to validate the request timeout (int): timeout in seconds - max_retry (int): num of times we want to retry to send a create - topic request when the controller in not available Returns: CreateTopicResponse: response from the broker @@ -132,5 +129,3 @@ def delete_topics(self, topics, timeout): timeout=timeout, ) return self._send_request(request) - - From 8642096864e83890f641bb060f3a809c83e5ae1f Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 10 Jan 2018 16:42:47 -0800 Subject: [PATCH 7/9] add delete_topics unit test --- test/test_admin_client.py | 29 ++++++++++++++++++++++++++--- 1 file changed, 26 insertions(+), 3 deletions(-) diff --git a/test/test_admin_client.py b/test/test_admin_client.py index 4f5d89544..2b4776246 100644 --- a/test/test_admin_client.py +++ b/test/test_admin_client.py @@ -3,7 +3,7 @@ from kafka.client_async import KafkaClient from kafka.errors import BrokerNotAvailableError from kafka.protocol.metadata import MetadataResponse -from kafka.protocol.admin import CreateTopicsResponse +from kafka.protocol.admin import CreateTopicsResponse, DeleteTopicsResponse from kafka.admin_client import AdminClient from kafka.admin_client import NewTopic from kafka.structs import BrokerMetadata @@ -38,6 +38,11 @@ def topic_response(): 'topic',7,'timeout_exception' )]) +@pytest.fixture +def delete_response(): + return DeleteTopicsResponse[0]([( + 'topic',7 + )]) class TestTopicAdmin(): @@ -73,6 +78,24 @@ def test_create_topics( mock_least_loaded_node mock_kafka_client.send.return_value = Future() admin = AdminClient(mock_kafka_client) - response = admin.create_topics(mock_new_topics, False) + response = admin.create_topics(mock_new_topics, 0) assert response == topic_response - + + def delete_topics( + self, + mock_new_topics, + mock_least_loaded_node, + bootstrap_brokers, + delete_response, + metadata_response, + ): + mock_kafka_client = mock.Mock() + mock_kafka_client.poll = \ + mock.Mock(side_effect=[metadata_response, delete_response]) + mock_kafka_client.ready.return_value = True + mock_kafka_client.least_loaded_node.return_value = \ + mock_least_loaded_node + mock_kafka_client.send.return_value = Future() + admin = AdminClient(mock_kafka_client) + response = admin.delete_topics(mock_new_topics, 0) + assert response == delete_response From 8db92934697c681d2b9aa9e00a2b1c4181f05ac8 Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Wed, 10 Jan 2018 17:46:54 -0800 Subject: [PATCH 8/9] add test_ to delete_topic and add comment in integration test --- test/test_admin_client.py | 2 +- test/test_admin_client_integration.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/test/test_admin_client.py b/test/test_admin_client.py index 2b4776246..ccbd16dd4 100644 --- a/test/test_admin_client.py +++ b/test/test_admin_client.py @@ -81,7 +81,7 @@ def test_create_topics( response = admin.create_topics(mock_new_topics, 0) assert response == topic_response - def delete_topics( + def test_delete_topics( self, mock_new_topics, mock_least_loaded_node, diff --git a/test/test_admin_client_integration.py b/test/test_admin_client_integration.py index 3a1a748a5..5b1c73589 100644 --- a/test/test_admin_client_integration.py +++ b/test/test_admin_client_integration.py @@ -34,6 +34,10 @@ def test_create_delete_topics(self): ) metadata_request = MetadataRequest[1]() response = admin.create_topics(topics=[topic], timeout=1) + # Error code 7 means that RequestTimedOut but we can safely assume + # that topic is created or will be created eventually. + # see this https://cwiki.apache.org/confluence/display/KAFKA/ + # KIP-4+-+Command+line+and+centralized+administrative+operations self.assertTrue( response[0].topic_error_codes[0][1] == 0 or response[0].topic_error_codes[0][1] == 7 From 0deaff41941d4fd5908a52cec2e139c420ff996a Mon Sep 17 00:00:00 2001 From: Ashutosh Singh Date: Thu, 11 Jan 2018 11:54:12 -0800 Subject: [PATCH 9/9] fix docstring --- kafka/admin_client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/admin_client.py b/kafka/admin_client.py index 67152f9c2..30147bc1e 100644 --- a/kafka/admin_client.py +++ b/kafka/admin_client.py @@ -121,7 +121,7 @@ def delete_topics(self, topics, timeout): topics (list of topic names): Topics to delete timeout (int): The requested timeout for this operation Raises: - NodeNotReadyError: if retry exceeds max_retry + NodeNotReadyError: if controller is not ready """ request = self.delete_topics_request(