Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Create admin client #44

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 9 commits into from
Jan 11, 2018
Merged
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
131 changes: 131 additions & 0 deletions kafka/admin_client.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
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 "<name>:{}, <num_partitions>:{}, <replication_factor>:{}" \
"<replica_assignments>:{}, <configs>:{}".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:
topics (list of NewTopic): A list containing new
topics to be created
timeout (int): timeout in seconds

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 controller is not ready
"""

request = self.delete_topics_request(
topics=topics,
timeout=timeout,
)
return self._send_request(request)
2 changes: 1 addition & 1 deletion kafka/protocol/admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand Down
6 changes: 6 additions & 0 deletions run_itest.sh
Original file line number Diff line number Diff line change
Expand Up @@ -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
16 changes: 1 addition & 15 deletions test/fixtures.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
101 changes: 101 additions & 0 deletions test/test_admin_client.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
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, DeleteTopicsResponse
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'
)])

@pytest.fixture
def delete_response():
return DeleteTopicsResponse[0]([(
'topic',7
)])

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, 0)
assert response == topic_response

def test_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
50 changes: 50 additions & 0 deletions test/test_admin_client_integration.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
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)
# 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
)
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
)
2 changes: 2 additions & 0 deletions test/testutil.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
from . import unittest

from kafka import SimpleClient
from kafka.client_async import KafkaClient
from kafka.structs import OffsetRequestPayload

__all__ = [
Expand Down Expand Up @@ -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)

Expand Down
10 changes: 3 additions & 7 deletions tox.ini
Original file line number Diff line number Diff line change
Expand Up @@ -12,12 +12,13 @@ deps =
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 =
Expand All @@ -30,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}
Expand Down