@@ -704,6 +704,56 @@ static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args,
704
704
}
705
705
706
706
707
+ static PyObject * Consumer_offsets_for_times (Handle * self , PyObject * args ,
708
+ PyObject * kwargs ) {
709
+ #if RD_KAFKA_VERSION < 0x000b0000
710
+ PyErr_Format (PyExc_NotImplementedError ,
711
+ "Consumer offsets_for_times require "
712
+ "confluent-kafka-python built for librdkafka "
713
+ "version >=v0.11.0 (librdkafka runtime 0x%x, "
714
+ "buildtime 0x%x)" ,
715
+ rd_kafka_version (), RD_KAFKA_VERSION );
716
+ return NULL ;
717
+ #else
718
+
719
+ PyObject * plist ;
720
+ double tmout = -1.0f ;
721
+ rd_kafka_topic_partition_list_t * c_parts ;
722
+ rd_kafka_resp_err_t err ;
723
+ static char * kws [] = { "partitions" , "timeout" , NULL };
724
+
725
+ if (!self -> rk ) {
726
+ PyErr_SetString (PyExc_RuntimeError ,
727
+ "Consumer closed" );
728
+ return NULL ;
729
+ }
730
+
731
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "O|d" , kws ,
732
+ & plist , & tmout ))
733
+ return NULL ;
734
+
735
+ if (!(c_parts = py_to_c_parts (plist )))
736
+ return NULL ;
737
+
738
+ err = rd_kafka_offsets_for_times (self -> rk ,
739
+ c_parts ,
740
+ tmout >= 0 ? (int )(tmout * 1000.0f ) : -1 );
741
+
742
+ if (err ) {
743
+ rd_kafka_topic_partition_list_destroy (c_parts );
744
+ cfl_PyErr_Format (err ,
745
+ "Failed to get offsets: %s" ,
746
+ rd_kafka_err2str (err ));
747
+ return NULL ;
748
+ }
749
+
750
+ plist = c_parts_to_py (c_parts );
751
+ rd_kafka_topic_partition_list_destroy (c_parts );
752
+
753
+ return plist ;
754
+ #endif
755
+ }
756
+
707
757
708
758
static PyObject * Consumer_poll (Handle * self , PyObject * args ,
709
759
PyObject * kwargs ) {
@@ -1030,6 +1080,24 @@ static PyMethodDef Consumer_methods[] = {
1030
1080
" :raises: RuntimeError if called on a closed consumer\n"
1031
1081
"\n"
1032
1082
},
1083
+ { "offsets_for_times" , (PyCFunction )Consumer_offsets_for_times ,
1084
+ METH_VARARGS |METH_KEYWORDS ,
1085
+ ".. py:function:: offsets_for_times(partitions, [timeout=None])\n"
1086
+ "\n"
1087
+ " offsets_for_times looks up offsets by timestamp for the given partitions.\n"
1088
+ "\n"
1089
+ " The returned offsets for each partition is the earliest offset whose\n"
1090
+ " timestamp is greater than or equal to the given timestamp in the\n"
1091
+ " corresponding partition.\n"
1092
+ "\n"
1093
+ " :param list(TopicPartition) partitions: topic+partitions with timestamps in the TopicPartition.offset field."
1094
+ " :param float timeout: Request timeout.\n"
1095
+ " :returns: list of topic+partition with offset field set and possibly error set\n"
1096
+ " :rtype: list(TopicPartition)\n"
1097
+ " :raises: KafkaException\n"
1098
+ " :raises: RuntimeError if called on a closed consumer\n"
1099
+ "\n"
1100
+ },
1033
1101
{ "close" , (PyCFunction )Consumer_close , METH_NOARGS ,
1034
1102
"\n"
1035
1103
" Close down and terminate the Kafka Consumer.\n"
0 commit comments