Skip to content

Commit 312bd1f

Browse files
authored
Merge pull request #61 from confluentinc/PR50_msg_timestamps
PR #50 message timestamps - additional docs, etc
2 parents 745b6f3 + 11b8a20 commit 312bd1f

File tree

7 files changed

+90
-28
lines changed

7 files changed

+90
-28
lines changed

README.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@ Tests
115115

116116
**Run integration tests:**
117117

118-
$ examples/integration_test.py <kafka-broker>
118+
$ examples/integration_test.py <kafka-broker> [<test-topic>]
119119

120120
**WARNING**: These tests require an active Kafka cluster and will make use of a topic named 'test'.
121121

confluent_kafka/src/confluent_kafka.c

Lines changed: 36 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -326,6 +326,13 @@ static PyObject *Message_offset (Message *self, PyObject *ignore) {
326326
}
327327

328328

329+
static PyObject *Message_timestamp (Message *self, PyObject *ignore) {
330+
return Py_BuildValue("iL",
331+
self->tstype,
332+
self->timestamp);
333+
}
334+
335+
329336
static PyMethodDef Message_methods[] = {
330337
{ "error", (PyCFunction)Message_error, METH_NOARGS,
331338
" The message object is also used to propagate errors and events, "
@@ -362,6 +369,28 @@ static PyMethodDef Message_methods[] = {
362369
" :rtype: int or None\n"
363370
"\n"
364371
},
372+
{ "timestamp", (PyCFunction)Message_timestamp, METH_NOARGS,
373+
" Retrieve timestamp type and timestamp from message.\n"
374+
" The timestamp type is one of:\n"
375+
" * :py:const:`TIMESTAMP_NOT_AVAILABLE`"
376+
" - Timestamps not supported by broker\n"
377+
" * :py:const:`TIMESTAMP_CREATE_TIME` "
378+
" - Message creation time (or source / producer time)\n"
379+
" * :py:const:`TIMESTAMP_LOG_APPEND_TIME` "
380+
" - Broker receive time\n"
381+
"\n"
382+
" The returned timestamp should be ignored if the timestamp type is "
383+
":py:const:`TIMESTAMP_NOT_AVAILABLE`.\n"
384+
"\n"
385+
" The timestamp is the number of milliseconds since the epoch (UTC).\n"
386+
"\n"
387+
" Timestamps require broker version 0.10.0.0 or later and \n"
388+
" ``{'api.version.request': True}`` configured on the client.\n"
389+
"\n"
390+
" :returns: tuple of message timestamp type, and timestamp.\n"
391+
" :rtype: (int, int)\n"
392+
"\n"
393+
},
365394
{ NULL }
366395
};
367396

@@ -441,7 +470,7 @@ PyTypeObject MessageType = {
441470
"An application must check with :py:func:`error()` to see if the "
442471
"object is a proper message (error() returns None) or an "
443472
"error/event.\n"
444-
"\n"
473+
"\n"
445474
"This class is not user-instantiable.\n"
446475
"\n"
447476
".. py:function:: len()\n"
@@ -495,6 +524,8 @@ PyObject *Message_new0 (const rd_kafka_message_t *rkm) {
495524
self->partition = rkm->partition;
496525
self->offset = rkm->offset;
497526

527+
self->timestamp = rd_kafka_message_timestamp(rkm, &self->tstype);
528+
498529
return (PyObject *)self;
499530
}
500531

@@ -1484,6 +1515,10 @@ static PyObject *_init_cimpl (void) {
14841515
Py_INCREF(KafkaException);
14851516
PyModule_AddObject(m, "KafkaException", KafkaException);
14861517

1518+
PyModule_AddIntConstant(m, "TIMESTAMP_NOT_AVAILABLE", RD_KAFKA_TIMESTAMP_NOT_AVAILABLE);
1519+
PyModule_AddIntConstant(m, "TIMESTAMP_CREATE_TIME", RD_KAFKA_TIMESTAMP_CREATE_TIME);
1520+
PyModule_AddIntConstant(m, "TIMESTAMP_LOG_APPEND_TIME", RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME);
1521+
14871522
return m;
14881523
}
14891524

confluent_kafka/src/confluent_kafka.h

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -220,6 +220,8 @@ typedef struct {
220220
PyObject *error;
221221
int32_t partition;
222222
int64_t offset;
223+
int64_t timestamp;
224+
rd_kafka_timestamp_type_t tstype;
223225
} Message;
224226

225227
extern PyTypeObject MessageType;

examples/integration_test.py

Lines changed: 38 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,16 @@
3333
with_progress = False
3434

3535
# Kafka bootstrap server(s)
36-
bootstrap_servers = 'localhost'
36+
bootstrap_servers = None
3737

38+
# Topic to use
39+
topic = 'test'
40+
41+
# API version requests are only implemented in Kafka broker >=0.10
42+
# but the client handles failed API version requests gracefully for older
43+
# versions as well, except for 0.9.0.x which will stall for about 10s
44+
# on each connect with this set to True.
45+
api_version_request = True
3846

3947
# global variable to be set by stats_cb call back function
4048
good_stats_cb_result = False
@@ -83,29 +91,30 @@ def verify_producer():
8391
# Producer config
8492
conf = {'bootstrap.servers': bootstrap_servers,
8593
'error_cb': error_cb,
94+
'api.version.request': api_version_request,
8695
'default.topic.config':{'produce.offset.report': True}}
8796

8897
# Create producer
8998
p = confluent_kafka.Producer(**conf)
9099
print('producer at %s' % p)
91100

92101
# Produce some messages
93-
p.produce('test', 'Hello Python!')
94-
p.produce('test', key='Just a key')
95-
p.produce('test', partition=1, value='Strictly for partition 1',
102+
p.produce(topic, 'Hello Python!')
103+
p.produce(topic, key='Just a key')
104+
p.produce(topic, partition=1, value='Strictly for partition 1',
96105
key='mykey')
97106

98107
# Produce more messages, now with delivery report callbacks in various forms.
99108
mydr = MyTestDr()
100-
p.produce('test', value='This one has a dr callback',
109+
p.produce(topic, value='This one has a dr callback',
101110
callback=mydr.delivery)
102-
p.produce('test', value='This one has a lambda',
111+
p.produce(topic, value='This one has a lambda',
103112
callback=lambda err, msg: MyTestDr._delivery(err, msg))
104-
p.produce('test', value='This one has neither')
113+
p.produce(topic, value='This one has neither')
105114

106115
# Produce even more messages
107116
for i in range(0, 10):
108-
p.produce('test', value='Message #%d' % i, key=str(i),
117+
p.produce(topic, value='Message #%d' % i, key=str(i),
109118
callback=mydr.delivery)
110119
p.poll(0)
111120

@@ -119,11 +128,11 @@ def verify_producer():
119128
def verify_producer_performance(with_dr_cb=True):
120129
""" Time how long it takes to produce and delivery X messages """
121130
conf = {'bootstrap.servers': bootstrap_servers,
131+
'api.version.request': api_version_request,
122132
'error_cb': error_cb}
123133

124134
p = confluent_kafka.Producer(**conf)
125135

126-
topic = 'test'
127136
msgcnt = 1000000
128137
msgsize = 100
129138
msg_pattern = 'test.py performance'
@@ -144,9 +153,9 @@ def verify_producer_performance(with_dr_cb=True):
144153
for i in range(0, msgcnt):
145154
try:
146155
if with_dr_cb:
147-
p.produce('test', value=msg_payload, callback=dr.delivery)
156+
p.produce(topic, value=msg_payload, callback=dr.delivery)
148157
else:
149-
p.produce('test', value=msg_payload)
158+
p.produce(topic, value=msg_payload)
150159
except BufferError as e:
151160
# Local queue is full (slow broker connection?)
152161
msgs_backpressure += 1
@@ -213,6 +222,7 @@ def verify_consumer():
213222
'group.id': 'test.py',
214223
'session.timeout.ms': 6000,
215224
'enable.auto.commit': False,
225+
'api.version.request': api_version_request,
216226
'on_commit': print_commit_result,
217227
'error_cb': error_cb,
218228
'default.topic.config': {
@@ -223,7 +233,7 @@ def verify_consumer():
223233
c = confluent_kafka.Consumer(**conf)
224234

225235
# Subscribe to a list of topics
226-
c.subscribe(["test"])
236+
c.subscribe([topic])
227237

228238
max_msgcnt = 100
229239
msgcnt = 0
@@ -245,10 +255,10 @@ def verify_consumer():
245255
print('Consumer error: %s: ignoring' % msg.error())
246256
break
247257

248-
if False:
249-
print('%s[%d]@%d: key=%s, value=%s' % \
250-
(msg.topic(), msg.partition(), msg.offset(),
251-
msg.key(), msg.value()))
258+
tstype, timestamp = msg.timestamp()
259+
print('%s[%d]@%d: key=%s, value=%s, tstype=%d, timestamp=%s' % \
260+
(msg.topic(), msg.partition(), msg.offset(),
261+
msg.key(), msg.value(), tstype, timestamp))
252262

253263
if (msg.offset() % 5) == 0:
254264
# Async commit
@@ -268,7 +278,7 @@ def verify_consumer():
268278

269279
# Start a new client and get the committed offsets
270280
c = confluent_kafka.Consumer(**conf)
271-
offsets = c.committed(list(map(lambda p: confluent_kafka.TopicPartition("test", p), range(0,3))))
281+
offsets = c.committed(list(map(lambda p: confluent_kafka.TopicPartition(topic, p), range(0,3))))
272282
for tp in offsets:
273283
print(tp)
274284

@@ -302,7 +312,7 @@ def my_on_revoke (consumer, partitions):
302312
print(' %s [%d] @ %d' % (p.topic, p.partition, p.offset))
303313
consumer.unassign()
304314

305-
c.subscribe(["test"], on_assign=my_on_assign, on_revoke=my_on_revoke)
315+
c.subscribe([topic], on_assign=my_on_assign, on_revoke=my_on_revoke)
306316

307317
max_msgcnt = 1000000
308318
bytecnt = 0
@@ -362,10 +372,11 @@ def verify_stats_cb():
362372
def stats_cb(stats_json_str):
363373
global good_stats_cb_result
364374
stats_json = json.loads(stats_json_str)
365-
if 'test' in stats_json['topics']:
366-
app_offset = stats_json['topics']['test']['partitions']['0']['app_offset']
375+
if topic in stats_json['topics']:
376+
app_offset = stats_json['topics'][topic]['partitions']['0']['app_offset']
367377
if app_offset > 0:
368-
print("# app_offset stats for topic test partition 0: %d" % app_offset)
378+
print("# app_offset stats for topic %s partition 0: %d" % \
379+
(topic, app_offset))
369380
good_stats_cb_result = True
370381

371382
conf = {'bootstrap.servers': bootstrap_servers,
@@ -379,7 +390,7 @@ def stats_cb(stats_json_str):
379390
}}
380391

381392
c = confluent_kafka.Consumer(**conf)
382-
c.subscribe(["test"])
393+
c.subscribe([topic])
383394

384395
max_msgcnt = 1000000
385396
bytecnt = 0
@@ -437,6 +448,11 @@ def stats_cb(stats_json_str):
437448

438449
if len(sys.argv) > 1:
439450
bootstrap_servers = sys.argv[1]
451+
if len(sys.argv) > 2:
452+
topic = sys.argv[2]
453+
else:
454+
print('Usage: %s <broker> [<topic>]' % sys.argv[0])
455+
sys.exit(1)
440456

441457
print('Using confluent_kafka module version %s (0x%x)' % confluent_kafka.version())
442458
print('Using librdkafka version %s (0x%x)' % confluent_kafka.libversion())

tests/test_Consumer.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
#!/usr/bin/env python
22

3-
from confluent_kafka import Consumer, TopicPartition, KafkaError, KafkaException
3+
from confluent_kafka import Consumer, TopicPartition, KafkaError, KafkaException, TIMESTAMP_NOT_AVAILABLE
44

55

66
def test_basic_api():
@@ -36,6 +36,9 @@ def dummy_assign_revoke (consumer, partitions):
3636
else:
3737
print('OK: consumed message')
3838

39+
if msg is not None:
40+
assert msg.timestamp() == (TIMESTAMP_NOT_AVAILABLE, -1)
41+
3942
partitions = list(map(lambda p: TopicPartition("test", p), range(0,100,3)))
4043
kc.assign(partitions)
4144

tests/test_docs.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,9 @@ def test_verify_docs():
2121
fails += 1
2222
elif not re.search(r':', d):
2323
print('Missing Doxygen tag for: %s (type %s)' % (n, type(o)))
24-
if not isinstance(o, ModuleType):
24+
# Ignore missing doc strings for the cimpl module itself and
25+
# integer constants (which can't have a doc string)
26+
if n != 'cimpl' and type(o) not in [int]:
2527
fails += 1
2628

2729
assert fails == 0
28-
29-

tests/test_enums.py

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,3 +7,9 @@ def test_enums():
77
KafkaError class without an instantiated object. """
88
print(confluent_kafka.KafkaError._NO_OFFSET)
99
print(confluent_kafka.KafkaError.REBALANCE_IN_PROGRESS)
10+
11+
def test_tstype_enums():
12+
""" Make sure librdkafka tstype enums are available. """
13+
assert confluent_kafka.TIMESTAMP_NOT_AVAILABLE == 0
14+
assert confluent_kafka.TIMESTAMP_CREATE_TIME == 1
15+
assert confluent_kafka.TIMESTAMP_LOG_APPEND_TIME == 2

0 commit comments

Comments
 (0)