Skip to content

Commit 99f8eac

Browse files
committed
kafkatest: add --message-create-time and --producer.config support
1 parent e226538 commit 99f8eac

File tree

3 files changed

+56
-23
lines changed

3 files changed

+56
-23
lines changed

confluent_kafka/kafkatest/verifiable_client.py

Lines changed: 33 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -70,17 +70,44 @@ def set_config(conf, args):
7070
for n, v in args.iteritems():
7171
if v is None:
7272
continue
73-
# Things to ignore
74-
if '.' not in n:
73+
74+
if not n.startswith('conf_'):
7575
# App config, skip
7676
continue
77-
if n.startswith('topic.'):
78-
# Set "topic.<...>" properties on default topic conf dict
79-
conf['default.topic.config'][n[6:]] = v
80-
elif n == 'partition.assignment.strategy':
77+
78+
n = n[5:] # Remove conf_ prefix
79+
80+
if n == 'partition.assignment.strategy':
8181
# Convert Java class name to config value.
8282
# "org.apache.kafka.clients.consumer.RangeAssignor" -> "range"
8383
conf[n] = re.sub(r'org.apache.kafka.clients.consumer.(\w+)Assignor',
8484
lambda x: x.group(1).lower(), v)
85+
# Handle known Java properties to librdkafka properties.
86+
elif n == 'enable.idempotence':
87+
# Ignore idempotence for now, best-effortly.
88+
sys.stderr.write('%% WARN: Ignoring unsupported %s=%s\n' % (n, v))
8589
else:
8690
conf[n] = v
91+
92+
@staticmethod
93+
def read_config_file (path):
94+
"""Read (java client) config file and return dict with properties"""
95+
conf = {}
96+
97+
with open(path, 'r') as f:
98+
for line in f:
99+
line = line.strip()
100+
101+
if line.startswith('#') or len(line) == 0:
102+
continue
103+
104+
fi = line.find('=')
105+
if fi < 1:
106+
raise Exception('%s: invalid line, no key=value pair: %s' % (path, line))
107+
108+
k = line[:fi]
109+
v = line[fi+1:]
110+
111+
conf[k] = v
112+
113+
return conf

confluent_kafka/kafkatest/verifiable_consumer.py

Lines changed: 12 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -242,30 +242,32 @@ def to_dict(self):
242242

243243
parser = argparse.ArgumentParser(description='Verifiable Python Consumer')
244244
parser.add_argument('--topic', action='append', type=str, required=True)
245-
parser.add_argument('--group-id', dest='group.id', required=True)
246-
parser.add_argument('--broker-list', dest='bootstrap.servers', required=True)
247-
parser.add_argument('--session-timeout', type=int, dest='session.timeout.ms', default=6000)
248-
parser.add_argument('--enable-autocommit', action='store_true', dest='enable.auto.commit', default=False)
245+
parser.add_argument('--group-id', dest='conf_group.id', required=True)
246+
parser.add_argument('--broker-list', dest='conf_bootstrap.servers', required=True)
247+
parser.add_argument('--session-timeout', type=int, dest='conf_session.timeout.ms', default=6000)
248+
parser.add_argument('--enable-autocommit', action='store_true', dest='conf_enable.auto.commit', default=False)
249249
parser.add_argument('--max-messages', type=int, dest='max_messages', default=-1)
250-
parser.add_argument('--assignment-strategy', dest='partition.assignment.strategy')
251-
parser.add_argument('--reset-policy', dest='topic.auto.offset.reset', default='earliest')
250+
parser.add_argument('--assignment-strategy', dest='conf_partition.assignment.strategy')
251+
parser.add_argument('--reset-policy', dest='conf_auto.offset.reset', default='earliest')
252252
parser.add_argument('--consumer.config', dest='consumer_config')
253253
parser.add_argument('-X', nargs=1, dest='extra_conf', action='append', help='Configuration property', default=[])
254254
args = vars(parser.parse_args())
255255

256256
conf = {'broker.version.fallback': '0.9.0',
257-
'default.topic.config': dict(),
258257
# Do explicit manual offset stores to avoid race conditions
259258
# where a message is consumed from librdkafka but not yet handled
260259
# by the Python code that keeps track of last consumed offset.
261260
'enable.auto.offset.store': False}
262261

263-
VerifiableClient.set_config(conf, args)
262+
if args.get('consumer_config', None) is not None:
263+
args.update(VerifiableClient.read_config_file(args['consumer_config']))
264+
265+
args.update([x[0].split('=') for x in args.get('extra_conf', [])])
264266

265-
conf.update([x[0].split('=') for x in args.get('extra_conf', [])])
267+
VerifiableClient.set_config(conf, args)
266268

267269
vc = VerifiableConsumer(conf)
268-
vc.use_auto_commit = args['enable.auto.commit']
270+
vc.use_auto_commit = args['conf_enable.auto.commit']
269271
vc.max_msgs = args['max_messages']
270272

271273
vc.dbg('Pid %d' % os.getpid())

confluent_kafka/kafkatest/verifiable_producer.py

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ def __init__(self, conf):
3232
"""
3333
super(VerifiableProducer, self).__init__(conf)
3434
self.conf['on_delivery'] = self.dr_cb
35-
self.conf['default.topic.config']['produce.offset.report'] = True
3635
self.producer = Producer(**self.conf)
3736
self.num_acked = 0
3837
self.num_sent = 0
@@ -64,20 +63,24 @@ def dr_cb(self, err, msg):
6463
parser = argparse.ArgumentParser(description='Verifiable Python Producer')
6564
parser.add_argument('--topic', type=str, required=True)
6665
parser.add_argument('--throughput', type=int, default=0)
67-
parser.add_argument('--broker-list', dest='bootstrap.servers', required=True)
66+
parser.add_argument('--broker-list', dest='conf_bootstrap.servers', required=True)
6867
parser.add_argument('--max-messages', type=int, dest='max_msgs', default=1000000) # avoid infinite
6968
parser.add_argument('--value-prefix', dest='value_prefix', type=str, default=None)
70-
parser.add_argument('--acks', type=int, dest='topic.request.required.acks', default=-1)
69+
parser.add_argument('--acks', type=int, dest='conf_request.required.acks', default=-1)
70+
parser.add_argument('--message-create-time', type=int, dest='create_time', default=0)
7171
parser.add_argument('--producer.config', dest='producer_config')
7272
parser.add_argument('-X', nargs=1, dest='extra_conf', action='append', help='Configuration property', default=[])
7373
args = vars(parser.parse_args())
7474

7575
conf = {'broker.version.fallback': '0.9.0',
76-
'default.topic.config': dict()}
76+
'produce.offset.report': True}
7777

78-
VerifiableClient.set_config(conf, args)
78+
if args.get('producer_config', None) is not None:
79+
args.update(VerifiableClient.read_config_file(args['producer_config']))
80+
81+
args.update([x[0].split('=') for x in args.get('extra_conf', [])])
7982

80-
conf.update([x[0].split('=') for x in args.get('extra_conf', [])])
83+
VerifiableClient.set_config(conf, args)
8184

8285
vp = VerifiableProducer(conf)
8386

@@ -104,7 +107,8 @@ def dr_cb(self, err, msg):
104107
t_end = time.time() + delay
105108
while vp.run:
106109
try:
107-
vp.producer.produce(topic, value=(value_fmt % i))
110+
vp.producer.produce(topic, value=(value_fmt % i),
111+
timestamp=args.get('create_time', 0))
108112
vp.num_sent += 1
109113
except KafkaException as e:
110114
vp.err('produce() #%d/%d failed: %s' % (i, vp.max_msgs, str(e)))

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy