@@ -232,25 +232,25 @@ static PyObject *Consumer_unassign (Handle *self, PyObject *ignore) {
232
232
}
233
233
234
234
static PyObject * Consumer_assignment (Handle * self , PyObject * args ,
235
- PyObject * kwargs ) {
236
-
237
- PyObject * plist ;
238
- rd_kafka_topic_partition_list_t * c_parts ;
239
- rd_kafka_resp_err_t err ;
240
-
241
- err = rd_kafka_assignment (self -> rk , & c_parts );
242
- if (err ) {
243
- cfl_PyErr_Format (err ,
244
- "Failed to get assignment: %s" ,
245
- rd_kafka_err2str (err ));
246
- return NULL ;
247
- }
235
+ PyObject * kwargs ) {
236
+
237
+ PyObject * plist ;
238
+ rd_kafka_topic_partition_list_t * c_parts ;
239
+ rd_kafka_resp_err_t err ;
240
+
241
+ err = rd_kafka_assignment (self -> rk , & c_parts );
242
+ if (err ) {
243
+ cfl_PyErr_Format (err ,
244
+ "Failed to get assignment: %s" ,
245
+ rd_kafka_err2str (err ));
246
+ return NULL ;
247
+ }
248
248
249
249
250
- plist = c_parts_to_py (c_parts );
251
- rd_kafka_topic_partition_list_destroy (c_parts );
250
+ plist = c_parts_to_py (c_parts );
251
+ rd_kafka_topic_partition_list_destroy (c_parts );
252
252
253
- return plist ;
253
+ return plist ;
254
254
}
255
255
256
256
@@ -392,33 +392,81 @@ static PyObject *Consumer_position (Handle *self, PyObject *args,
392
392
}
393
393
394
394
395
+ static PyObject * Consumer_get_watermark_offsets (Handle * self , PyObject * args ,
396
+ PyObject * kwargs ) {
397
+
398
+ TopicPartition * tp ;
399
+ rd_kafka_resp_err_t err ;
400
+ double tmout = -1.0f ;
401
+ int cached = 0 ;
402
+ int64_t low = RD_KAFKA_OFFSET_INVALID , high = RD_KAFKA_OFFSET_INVALID ;
403
+ static char * kws [] = { "partition" , "timeout" , "cached" , NULL };
404
+ PyObject * rtup ;
405
+
406
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|db" , kws ,
407
+ (PyObject * * )& tp , & tmout , & cached ))
408
+ return NULL ;
409
+
410
+
411
+ if (PyObject_Type ((PyObject * )tp ) != (PyObject * )& TopicPartitionType ) {
412
+ PyErr_Format (PyExc_TypeError ,
413
+ "expected %s" , TopicPartitionType .tp_name );
414
+ return NULL ;
415
+ }
416
+
417
+ if (cached ) {
418
+ err = rd_kafka_get_watermark_offsets (self -> rk ,
419
+ tp -> topic , tp -> partition ,
420
+ & low , & high );
421
+ } else {
422
+ err = rd_kafka_query_watermark_offsets (self -> rk ,
423
+ tp -> topic , tp -> partition ,
424
+ & low , & high ,
425
+ tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
426
+ }
427
+
428
+ if (err ) {
429
+ cfl_PyErr_Format (err ,
430
+ "Failed to get watermark offsets: %s" ,
431
+ rd_kafka_err2str (err ));
432
+ return NULL ;
433
+ }
434
+
435
+ rtup = PyTuple_New (2 );
436
+ PyTuple_SetItem (rtup , 0 , PyLong_FromLongLong (low ));
437
+ PyTuple_SetItem (rtup , 1 , PyLong_FromLongLong (high ));
438
+
439
+ return rtup ;
440
+ }
441
+
442
+
395
443
396
444
static PyObject * Consumer_poll (Handle * self , PyObject * args ,
397
- PyObject * kwargs ) {
398
- double tmout = -1.0f ;
399
- static char * kws [] = { "timeout" , NULL };
400
- rd_kafka_message_t * rkm ;
401
- PyObject * msgobj ;
402
- CallState cs ;
445
+ PyObject * kwargs ) {
446
+ double tmout = -1.0f ;
447
+ static char * kws [] = { "timeout" , NULL };
448
+ rd_kafka_message_t * rkm ;
449
+ PyObject * msgobj ;
450
+ CallState cs ;
403
451
404
- if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|d" , kws , & tmout ))
405
- return NULL ;
452
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "|d" , kws , & tmout ))
453
+ return NULL ;
406
454
407
- CallState_begin (self , & cs );
455
+ CallState_begin (self , & cs );
408
456
409
- rkm = rd_kafka_consumer_poll (self -> rk , tmout >= 0 ?
410
- (int )(tmout * 1000.0f ) : -1 );
457
+ rkm = rd_kafka_consumer_poll (self -> rk , tmout >= 0 ?
458
+ (int )(tmout * 1000.0f ) : -1 );
411
459
412
- if (!CallState_end (self , & cs ))
413
- return NULL ;
460
+ if (!CallState_end (self , & cs ))
461
+ return NULL ;
414
462
415
- if (!rkm )
416
- Py_RETURN_NONE ;
463
+ if (!rkm )
464
+ Py_RETURN_NONE ;
417
465
418
- msgobj = Message_new0 (rkm );
419
- rd_kafka_message_destroy (rkm );
466
+ msgobj = Message_new0 (rkm );
467
+ rd_kafka_message_destroy (rkm );
420
468
421
- return msgobj ;
469
+ return msgobj ;
422
470
}
423
471
424
472
@@ -511,17 +559,17 @@ static PyMethodDef Consumer_methods[] = {
511
559
" :raises: KafkaException\n"
512
560
"\n"
513
561
},
514
- { "assignment" , (PyCFunction )Consumer_assignment ,
515
- METH_VARARGS |METH_KEYWORDS ,
516
- ".. py:function:: assignment()\n"
517
- "\n"
518
- " Returns the current partition assignment.\n"
519
- "\n"
520
- " :returns: List of assigned topic+partitions.\n"
521
- " :rtype: list(TopicPartition)\n"
522
- " :raises: KafkaException\n"
523
- "\n"
524
- },
562
+ { "assignment" , (PyCFunction )Consumer_assignment ,
563
+ METH_VARARGS |METH_KEYWORDS ,
564
+ ".. py:function:: assignment()\n"
565
+ "\n"
566
+ " Returns the current partition assignment.\n"
567
+ "\n"
568
+ " :returns: List of assigned topic+partitions.\n"
569
+ " :rtype: list(TopicPartition)\n"
570
+ " :raises: KafkaException\n"
571
+ "\n"
572
+ },
525
573
{ "commit" , (PyCFunction )Consumer_commit , METH_VARARGS |METH_KEYWORDS ,
526
574
".. py:function:: commit([message=None], [offsets=None], [async=True])\n"
527
575
"\n"
@@ -566,6 +614,22 @@ static PyMethodDef Consumer_methods[] = {
566
614
" :raises: KafkaException\n"
567
615
"\n"
568
616
},
617
+ { "get_watermark_offsets" , (PyCFunction )Consumer_get_watermark_offsets ,
618
+ METH_VARARGS |METH_KEYWORDS ,
619
+ ".. py:function:: get_watermark_offsets(partition, [timeout=None], [cached=False])\n"
620
+ "\n"
621
+ " Retrieve low and high offsets for partition.\n"
622
+ "\n"
623
+ " :param TopicPartition partition: Topic+partition to return offsets for."
624
+ " :param float timeout: Request timeout (when cached=False).\n"
625
+ " :param bool cached: Instead of querying the broker used cached information. "
626
+ "Cached values: The low offset is updated periodically (if statistics.interval.ms is set) while "
627
+ "the high offset is updated on each message fetched from the broker for this partition."
628
+ " :returns: Tuple of (low,high) on success or None on timeout.\n"
629
+ " :rtype: tuple(int,int)\n"
630
+ " :raises: KafkaException\n"
631
+ "\n"
632
+ },
569
633
{ "close" , (PyCFunction )Consumer_close , METH_NOARGS ,
570
634
"\n"
571
635
" Close down and terminate the Kafka Consumer.\n"
0 commit comments