Skip to content
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
3 changes: 3 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@ test: build-integration
fixture: build-integration
python -m test.integration.fixtures kafka

sasl: build-integration
python -m test.integration.fixtures kafka --sasl

test-coverage: build-integration
pytest --cov=kafka --cov-report html test
@echo "open file://`pwd`/htmlcov/index.html"
Expand Down
25 changes: 9 additions & 16 deletions kafka/cli/admin/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,27 +8,14 @@
from .configs import ConfigsSubCommand
from .consumer_groups import ConsumerGroupsSubCommand
from .topics import TopicsSubCommand
from ..common import add_common_cli_args

def main_parser():
parser = argparse.ArgumentParser(
prog='python -m kafka.admin',
description='Kafka admin client',
)
parser.add_argument(
'-b', '--bootstrap-servers', type=str, action='append', required=True,
help='host:port for cluster bootstrap servers')
parser.add_argument(
'-c', '--extra-config', type=str, action='append',
help='additional configuration properties for admin client')
parser.add_argument(
'-l', '--log-level', type=str, default='CRITICAL',
help='logging level, passed to logging.basicConfig')
parser.add_argument(
'-L', '--enable-logger', type=str, action='append',
help='enable a specific logger. Can be provided multiple times. If not provided, all loggers are enabled')
parser.add_argument(
'-DL', '--disable-logger', type=str, action='append',
help='disable a specific logger. Can be provided multiple times.')
add_common_cli_args(parser)
parser.add_argument(
'-f', '--format', type=str, default='raw',
help='output format: raw|json')
Expand Down Expand Up @@ -84,7 +71,13 @@ def run_cli(args=None):
logger = logging.getLogger(__name__)

kwargs = build_kwargs(config.extra_config)
client = KafkaAdminClient(bootstrap_servers=config.bootstrap_servers, **kwargs)
client = KafkaAdminClient(
bootstrap_servers=config.bootstrap_servers,
security_protocol=config.security_protocol,
sasl_mechanism=config.sasl_mechanism,
sasl_plain_username=config.sasl_user,
sasl_plain_password=config.sasl_password,
**kwargs)
try:
result = config.command(client, config)
if config.format == 'raw':
Expand Down
27 changes: 27 additions & 0 deletions kafka/cli/common.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
def add_common_cli_args(parser):
connect_group = parser.add_argument_group('connection')
connect_group.add_argument(
'-b', '--bootstrap-servers', type=str, action='append', required=True,
help='host:port for cluster bootstrap server. Can be provided multiple times.')
connect_group.add_argument(
'-s', '--security-protocol', type=str, default='PLAINTEXT', help='PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL')
connect_group.add_argument(
'-m', '--sasl-mechanism', type=str, default='PLAIN', help='PLAIN, GSSAPI, OAUTHBEARER, SCRAM-SHA-256, SCRAM-SHA-512')
connect_group.add_argument(
'-u', '--sasl-user', type=str)
connect_group.add_argument(
'-p', '--sasl-password', type=str)
logging_group = parser.add_argument_group('logging')
logging_group.add_argument(
'-l', '--log-level', type=str, default='CRITICAL',
help='logging level, passed to logging.basicConfig')
logging_group.add_argument(
'-L', '--enable-logger', type=str, action='append',
help='enable a specific logger. Can be provided multiple times. If not provided, all loggers are enabled')
logging_group.add_argument(
'-DL', '--disable-logger', type=str, action='append',
help='disable a specific logger. Can be provided multiple times.')
extended_group = parser.add_argument_group('extended')
extended_group.add_argument(
'-c', '--extra-config', type=str, action='append',
help='additional configuration properties for client in "key=val" format. Can be provided multiple times.')
17 changes: 2 additions & 15 deletions kafka/cli/consumer/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,34 +3,21 @@

from kafka import KafkaConsumer
from kafka.errors import KafkaError
from ..common import add_common_cli_args


def main_parser():
parser = argparse.ArgumentParser(
prog='python -m kafka.consumer',
description='Kafka console consumer',
)
parser.add_argument(
'-b', '--bootstrap-servers', type=str, action='append', required=True,
help='host:port for cluster bootstrap server. Can be provided multiple times.')
add_common_cli_args(parser)
parser.add_argument(
'-t', '--topic', type=str, action='append', dest='topics', required=True,
help='subscribe to topic')
parser.add_argument(
'-g', '--group', type=str, required=True,
help='consumer group')
parser.add_argument(
'-c', '--extra-config', type=str, action='append',
help='additional configuration properties for kafka consumer')
parser.add_argument(
'-l', '--log-level', type=str, default='CRITICAL',
help='logging level, passed to logging.basicConfig')
parser.add_argument(
'-L', '--enable-logger', type=str, action='append',
help='enable a specific logger. Can be provided multiple times. If not provided, all loggers are enabled')
parser.add_argument(
'-DL', '--disable-logger', type=str, action='append',
help='disable a specific logger. Can be provided multiple times.')
parser.add_argument(
'-f', '--format', type=str, default='str',
help='output format: str|raw|full')
Expand Down
17 changes: 2 additions & 15 deletions kafka/cli/producer/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,31 +3,18 @@
import sys

from kafka import KafkaProducer
from ..common import add_common_cli_args


def main_parser():
parser = argparse.ArgumentParser(
prog='python -m kafka.producer',
description='Kafka console producer',
)
parser.add_argument(
'-b', '--bootstrap-servers', type=str, action='append', required=True,
help='host:port for cluster bootstrap servers')
add_common_cli_args(parser)
parser.add_argument(
'-t', '--topic', type=str, required=True,
help='publish to topic')
parser.add_argument(
'-c', '--extra-config', type=str, action='append',
help='additional configuration properties for kafka producer')
parser.add_argument(
'-l', '--log-level', type=str, default='CRITICAL',
help='logging level, passed to logging.basicConfig')
parser.add_argument(
'-L', '--enable-logger', type=str, action='append',
help='enable a specific logger. Can be provided multiple times. If not provided, all loggers are enabled')
parser.add_argument(
'-DL', '--disable-logger', type=str, action='append',
help='disable a specific logger. Can be provided multiple times.')
parser.add_argument(
'--encoding', type=str, default='utf-8',
help='byte encoding for produced messages')
Expand Down
2 changes: 0 additions & 2 deletions test/integration/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@ def factory(**broker_params):
broker = KafkaFixture.instance(node_id, **params)
_brokers.append(broker)
return broker

yield factory

zks = set()
Expand All @@ -66,7 +65,6 @@ def factory(**broker_params):
zk.close()



@pytest.fixture
def kafka_client(kafka_broker, request):
"""Return a KafkaClient fixture"""
Expand Down
10 changes: 7 additions & 3 deletions test/integration/fixtures.py
Original file line number Diff line number Diff line change
Expand Up @@ -727,9 +727,13 @@ def get_api_versions():
zk.close()


def run_brokers():
def run_brokers(args=()):
logging.basicConfig(level=logging.ERROR)
k = KafkaFixture.instance(0)
params = {}
if len(args) == 1 and args[0] == '--sasl':
params['transport'] = "SASL_PLAINTEXT"
params['sasl_mechanism'] = 'SCRAM-SHA-512'
k = KafkaFixture.instance(0, **params)
zk = k.zookeeper

print("Kafka", k.kafka_version, "running on port:", k.port)
Expand All @@ -752,7 +756,7 @@ def run_brokers():
if cmd == 'get_api_versions':
get_api_versions()
elif cmd == 'kafka':
run_brokers()
run_brokers(sys.argv[2:])
else:
print("Unknown cmd: %s", cmd)
exit(1)
Loading