@@ -242,7 +242,49 @@ int32_t Producer_partitioner_cb (const rd_kafka_topic_t *rkt,
242
242
}
243
243
244
244
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 ;
245
277
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
246
288
247
289
248
290
static PyObject * Producer_produce (Handle * self , PyObject * args ,
@@ -251,7 +293,8 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
251
293
int value_len = 0 , key_len = 0 ;
252
294
int partition = RD_KAFKA_PARTITION_UA ;
253
295
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 ;
255
298
struct Producer_msgstate * msgstate ;
256
299
static char * kws [] = { "topic" ,
257
300
"value" ,
@@ -260,25 +303,31 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
260
303
"callback" ,
261
304
"on_delivery" , /* Alias */
262
305
"partitioner" ,
306
+ "timestamp" ,
263
307
NULL };
264
308
265
309
if (!PyArg_ParseTupleAndKeywords (args , kwargs ,
266
- "s|z#z#iOOO" , kws ,
310
+ "s|z#z#iOOOL"
311
+ , kws ,
267
312
& topic , & value , & value_len ,
268
313
& key , & key_len , & partition ,
269
- & dr_cb , & dr_cb2 , & partitioner_cb ))
314
+ & dr_cb , & dr_cb2 , & partitioner_cb ,
315
+ & timestamp ))
270
316
return NULL ;
271
317
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
+
272
328
if (dr_cb2 && !dr_cb ) /* Alias */
273
329
dr_cb = dr_cb2 ;
274
330
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
-
282
331
if (!dr_cb || dr_cb == Py_None )
283
332
dr_cb = self -> u .Producer .default_dr_cb ;
284
333
if (!partitioner_cb || partitioner_cb == Py_None )
@@ -288,27 +337,34 @@ static PyObject *Producer_produce (Handle *self, PyObject *args,
288
337
* are wanted. */
289
338
msgstate = Producer_msgstate_new (self , dr_cb , partitioner_cb );
290
339
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 ) {
297
355
if (msgstate )
298
356
Producer_msgstate_destroy (msgstate );
299
357
300
358
if (err == RD_KAFKA_RESP_ERR__QUEUE_FULL )
301
359
PyErr_Format (PyExc_BufferError ,
302
360
"%s" , rd_kafka_err2str (err ));
303
- else
361
+ else
304
362
cfl_PyErr_Format (err ,
305
363
"Unable to produce message: %s" ,
306
364
rd_kafka_err2str (err ));
307
365
308
366
return NULL ;
309
367
}
310
-
311
- rd_kafka_topic_destroy (rkt );
312
368
313
369
Py_RETURN_NONE ;
314
370
}
@@ -364,7 +420,7 @@ static PyObject *Producer_flush (Handle *self, PyObject *ignore) {
364
420
static PyMethodDef Producer_methods [] = {
365
421
{ "produce" , (PyCFunction )Producer_produce ,
366
422
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"
368
424
"\n"
369
425
" Produce message to topic.\n"
370
426
" This is an asynchronous operation, an application may use the "
@@ -380,11 +436,13 @@ static PyMethodDef Producer_methods[] = {
380
436
" :param func on_delivery(err,msg): Delivery report callback to call "
381
437
"(from :py:func:`poll()` or :py:func:`flush()`) on successful or "
382
438
"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"
383
440
"\n"
384
441
" :rtype: None\n"
385
442
" :raises BufferError: if the internal producer message queue is "
386
443
"full (``queue.buffering.max.messages`` exceeded)\n"
387
444
" :raises KafkaException: for other errors, see exception code\n"
445
+ " :raises NotImplementedError: if timestamp is specified without underlying library support.\n"
388
446
"\n"
389
447
},
390
448
0 commit comments