@@ -325,6 +325,83 @@ static PyObject *Consumer_commit (Handle *self, PyObject *args,
325
325
326
326
327
327
328
+ static PyObject * Consumer_store_offsets (Handle * self , PyObject * args ,
329
+ PyObject * kwargs ) {
330
+
331
+ rd_kafka_resp_err_t err ;
332
+ PyObject * msg = NULL , * offsets = NULL ;
333
+ rd_kafka_topic_partition_list_t * c_offsets ;
334
+ static char * kws [] = { "message" , "offsets" , NULL };
335
+
336
+ #if RD_KAFKA_VERSION < 0x000b0000
337
+ PyErr_Format (PyExc_NotImplementedError ,
338
+ "Consumer store_offsets require "
339
+ "confluent-kafka-python built for librdkafka "
340
+ "version >=v0.11.0 (librdkafka runtime 0x%x, "
341
+ "buildtime 0x%x)" ,
342
+ rd_kafka_version (), RD_KAFKA_VERSION );
343
+ return NULL ;
344
+ #endif
345
+
346
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|OO" , kws ,
347
+ & msg , & offsets ))
348
+ return NULL ;
349
+
350
+ if (msg && offsets ) {
351
+ PyErr_SetString (PyExc_ValueError ,
352
+ "message and offsets are mutually exclusive" );
353
+ return NULL ;
354
+ }
355
+
356
+ if (!msg && !offsets ) {
357
+ PyErr_SetString (PyExc_ValueError ,
358
+ "expected either message or offsets" );
359
+ return NULL ;
360
+ }
361
+
362
+ if (offsets ) {
363
+
364
+ if (!(c_offsets = py_to_c_parts (offsets )))
365
+ return NULL ;
366
+ } else {
367
+ Message * m ;
368
+ PyObject * uo8 ;
369
+
370
+ if (PyObject_Type ((PyObject * )msg ) !=
371
+ (PyObject * )& MessageType ) {
372
+ PyErr_Format (PyExc_TypeError ,
373
+ "expected %s" , MessageType .tp_name );
374
+ return NULL ;
375
+ }
376
+
377
+ m = (Message * )msg ;
378
+
379
+ c_offsets = rd_kafka_topic_partition_list_new (1 );
380
+ rd_kafka_topic_partition_list_add (
381
+ c_offsets , cfl_PyUnistr_AsUTF8 (m -> topic , & uo8 ),
382
+ m -> partition )-> offset = m -> offset + 1 ;
383
+ Py_XDECREF (uo8 );
384
+ }
385
+
386
+
387
+ err = rd_kafka_offsets_store (self -> rk , c_offsets );
388
+
389
+ if (c_offsets )
390
+ rd_kafka_topic_partition_list_destroy (c_offsets );
391
+
392
+
393
+
394
+ if (err ) {
395
+ cfl_PyErr_Format (err ,
396
+ "StoreOffsets failed: %s" , rd_kafka_err2str (err ));
397
+ return NULL ;
398
+ }
399
+
400
+ Py_RETURN_NONE ;
401
+ }
402
+
403
+
404
+
328
405
static PyObject * Consumer_committed (Handle * self , PyObject * args ,
329
406
PyObject * kwargs ) {
330
407
@@ -570,6 +647,22 @@ static PyMethodDef Consumer_methods[] = {
570
647
" :raises: KafkaException\n"
571
648
"\n"
572
649
},
650
+ { "store_offsets" , (PyCFunction )Consumer_store_offsets , METH_VARARGS |METH_KEYWORDS ,
651
+ ".. py:function:: store_offsets([message=None], [offsets=None])\n"
652
+ "\n"
653
+ " Store offsets for a message or a list of offsets.\n"
654
+ "\n"
655
+ " ``message`` and ``offsets`` are mutually exclusive. "
656
+ "The stored offsets will be committed according to 'auto.commit.interval.ms' or manual "
657
+ "offset-less :py:meth:`commit`. "
658
+ "Note that 'enable.auto.offset.store' must be set to False when using this API.\n"
659
+ "\n"
660
+ " :param confluent_kafka.Message message: Store message's offset+1.\n"
661
+ " :param list(TopicPartition) offsets: List of topic+partitions+offsets to store.\n"
662
+ " :rtype: None\n"
663
+ " :raises: KafkaException\n"
664
+ "\n"
665
+ },
573
666
{ "commit" , (PyCFunction )Consumer_commit , METH_VARARGS |METH_KEYWORDS ,
574
667
".. py:function:: commit([message=None], [offsets=None], [async=True])\n"
575
668
"\n"
0 commit comments