Skip to content

Commit 821a6a6

Browse files
committed
Add timestamp= arg to produce()
Requires librdkafka >= 0.9.3 at build- and run-time
1 parent 956ff94 commit 821a6a6

File tree

4 files changed

+114
-25
lines changed

4 files changed

+114
-25
lines changed

confluent_kafka/src/Producer.c

Lines changed: 78 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -242,7 +242,49 @@ int32_t Producer_partitioner_cb (const rd_kafka_topic_t *rkt,
242242
}
243243

244244

245+
#if HAVE_PRODUCEV
246+
static rd_kafka_resp_err_t
247+
Producer_producev (Handle *self,
248+
const char *topic, int32_t partition,
249+
const void *value, size_t value_len,
250+
const void *key, size_t key_len,
251+
void *opaque, int64_t timestamp) {
252+
253+
return rd_kafka_producev(self->rk,
254+
RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY),
255+
RD_KAFKA_V_TOPIC(topic),
256+
RD_KAFKA_V_PARTITION(partition),
257+
RD_KAFKA_V_KEY(key, (size_t)key_len),
258+
RD_KAFKA_V_VALUE((void *)value,
259+
(size_t)value_len),
260+
RD_KAFKA_V_TIMESTAMP(timestamp),
261+
RD_KAFKA_V_OPAQUE(opaque),
262+
RD_KAFKA_V_END);
263+
}
264+
#else
265+
266+
static rd_kafka_resp_err_t
267+
Producer_produce0 (Handle *self,
268+
const char *topic, int32_t partition,
269+
const void *value, size_t value_len,
270+
const void *key, size_t key_len,
271+
void *opaque) {
272+
rd_kafka_topic_t *rkt;
273+
rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
274+
275+
if (!(rkt = rd_kafka_topic_new(self->rk, topic, NULL)))
276+
return RD_KAFKA_RESP_ERR__INVALID_ARG;
245277

278+
if (rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY,
279+
(void *)value, value_len,
280+
(void *)key, key_len, opaque) == -1)
281+
err = rd_kafka_last_error();
282+
283+
rd_kafka_topic_destroy(rkt);
284+
285+
return err;
286+
}
287+
#endif
246288

247289

248290
static PyObject *Producer_produce (Handle *self, PyObject *args,
@@ -251,7 +293,8 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
251293
int value_len = 0, key_len = 0;
252294
int partition = RD_KAFKA_PARTITION_UA;
253295
PyObject *dr_cb = NULL, *dr_cb2 = NULL, *partitioner_cb = NULL;
254-
rd_kafka_topic_t *rkt;
296+
long long timestamp = 0;
297+
rd_kafka_resp_err_t err;
255298
struct Producer_msgstate *msgstate;
256299
static char *kws[] = { "topic",
257300
"value",
@@ -260,25 +303,31 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
260303
"callback",
261304
"on_delivery", /* Alias */
262305
"partitioner",
306+
"timestamp",
263307
NULL };
264308

265309
if (!PyArg_ParseTupleAndKeywords(args, kwargs,
266-
"s|z#z#iOOO", kws,
310+
"s|z#z#iOOOL"
311+
, kws,
267312
&topic, &value, &value_len,
268313
&key, &key_len, &partition,
269-
&dr_cb, &dr_cb2, &partitioner_cb))
314+
&dr_cb, &dr_cb2, &partitioner_cb,
315+
&timestamp))
270316
return NULL;
271317

318+
#if !HAVE_PRODUCEV
319+
if (timestamp) {
320+
PyErr_Format(PyExc_NotImplementedError,
321+
"Producer timestamps require librdkafka "
322+
"version >=v0.9.3 (currently on %s)",
323+
rd_kafka_version_str());
324+
return NULL;
325+
}
326+
#endif
327+
272328
if (dr_cb2 && !dr_cb) /* Alias */
273329
dr_cb = dr_cb2;
274330

275-
if (!(rkt = rd_kafka_topic_new(self->rk, topic, NULL))) {
276-
cfl_PyErr_Format(rd_kafka_last_error(),
277-
"Unable to create topic object: %s",
278-
rd_kafka_err2str(rd_kafka_last_error()));
279-
return NULL;
280-
}
281-
282331
if (!dr_cb || dr_cb == Py_None)
283332
dr_cb = self->u.Producer.default_dr_cb;
284333
if (!partitioner_cb || partitioner_cb == Py_None)
@@ -288,27 +337,34 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
288337
* are wanted. */
289338
msgstate = Producer_msgstate_new(self, dr_cb, partitioner_cb);
290339

291-
/* Produce message */
292-
if (rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY,
293-
(void *)value, value_len,
294-
(void *)key, key_len, msgstate) == -1) {
295-
rd_kafka_resp_err_t err = rd_kafka_last_error();
296-
340+
/* Produce message */
341+
#if HAVE_PRODUCEV
342+
err = Producer_producev(self, topic, partition,
343+
value, value_len,
344+
key, key_len,
345+
msgstate, timestamp);
346+
#else
347+
err = Producer_produce0(self, topic, partition,
348+
value, value_len,
349+
key, key_len,
350+
msgstate);
351+
352+
#endif
353+
354+
if (err) {
297355
if (msgstate)
298356
Producer_msgstate_destroy(msgstate);
299357

300358
if (err == RD_KAFKA_RESP_ERR__QUEUE_FULL)
301359
PyErr_Format(PyExc_BufferError,
302360
"%s", rd_kafka_err2str(err));
303-
else
361+
else
304362
cfl_PyErr_Format(err,
305363
"Unable to produce message: %s",
306364
rd_kafka_err2str(err));
307365

308366
return NULL;
309367
}
310-
311-
rd_kafka_topic_destroy(rkt);
312368

313369
Py_RETURN_NONE;
314370
}
@@ -364,7 +420,7 @@ static PyObject *Producer_flush (Handle *self, PyObject *ignore) {
364420
static PyMethodDef Producer_methods[] = {
365421
{ "produce", (PyCFunction)Producer_produce,
366422
METH_VARARGS|METH_KEYWORDS,
367-
".. py:function:: produce(topic, [value], [key], [partition], [callback])\n"
423+
".. py:function:: produce(topic, [value], [key], [partition], [on_delivery], [timestamp])\n"
368424
"\n"
369425
" Produce message to topic.\n"
370426
" This is an asynchronous operation, an application may use the "
@@ -380,11 +436,13 @@ static PyMethodDef Producer_methods[] = {
380436
" :param func on_delivery(err,msg): Delivery report callback to call "
381437
"(from :py:func:`poll()` or :py:func:`flush()`) on successful or "
382438
"failed delivery\n"
439+
" :param int timestamp: Message timestamp (CreateTime) in microseconds since epoch UTC (requires librdkafka >= v0.9.3, api.version.request=true, and broker >= 0.10.0.0). Default value is current time.\n"
383440
"\n"
384441
" :rtype: None\n"
385442
" :raises BufferError: if the internal producer message queue is "
386443
"full (``queue.buffering.max.messages`` exceeded)\n"
387444
" :raises KafkaException: for other errors, see exception code\n"
445+
" :raises NotImplementedError: if timestamp is specified without underlying library support.\n"
388446
"\n"
389447
},
390448

confluent_kafka/src/confluent_kafka.h

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,14 @@
2626
#endif
2727

2828

29+
/**
30+
* librdkafka feature detection
31+
*/
32+
#if RD_KAFKA_VERSION >= 0x00090300
33+
#define HAVE_PRODUCEV 1 /* rd_kafka_producev() */
34+
#endif
35+
36+
2937

3038
/****************************************************************************
3139
*

examples/integration_test.py

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -112,6 +112,13 @@ def verify_producer():
112112
callback=lambda err, msg: MyTestDr._delivery(err, msg))
113113
p.produce(topic, value='This one has neither')
114114

115+
# Try producing with a timestamp
116+
try:
117+
p.produce(topic, value='with a timestamp', timestamp=123456789000)
118+
except NotImplementedError:
119+
if confluent_kafka.libversion()[1] >= 0x00090300:
120+
raise
121+
115122
# Produce even more messages
116123
for i in range(0, 10):
117124
p.produce(topic, value='Message #%d' % i, key=str(i),

tests/test_Producer.py

Lines changed: 21 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,10 @@
11
#!/usr/bin/env python
22

3-
from confluent_kafka import Producer, KafkaError, KafkaException
3+
from confluent_kafka import Producer, KafkaError, KafkaException, libversion
4+
import pytest
45

6+
def error_cb (err):
7+
print('error_cb', err)
58

69
def test_basic_api():
710
""" Basic API tests, these wont really do anything since there is no
@@ -12,10 +15,6 @@ def test_basic_api():
1215
except TypeError as e:
1316
assert str(e) == "expected configuration dict"
1417

15-
16-
def error_cb (err):
17-
print('error_cb', err)
18-
1918
p = Producer({'socket.timeout.ms':10,
2019
'error_cb': error_cb,
2120
'default.topic.config': {'message.timeout.ms': 10}})
@@ -36,6 +35,23 @@ def on_delivery(err,msg):
3635
p.flush()
3736

3837

38+
def test_produce_timestamp():
39+
""" Test produce() with timestamp arg """
40+
p = Producer({'socket.timeout.ms':10,
41+
'error_cb': error_cb,
42+
'default.topic.config': {'message.timeout.ms': 10}})
43+
44+
# Requires librdkafka >=v0.9.3
45+
46+
try:
47+
p.produce('mytopic', timestamp=1234567)
48+
except NotImplementedError:
49+
# Should only fail on non-supporting librdkafka
50+
if libversion()[1] >= 0x00090300:
51+
raise
52+
53+
p.flush()
54+
3955

4056
def test_subclassing():
4157
class SubProducer(Producer):

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