@@ -476,6 +476,13 @@ static PyObject *Message_offset (Message *self, PyObject *ignore) {
476
476
Py_RETURN_NONE ;
477
477
}
478
478
479
+ static PyObject * Message_leader_epoch (Message * self , PyObject * ignore ) {
480
+ if (self -> leader_epoch >= 0 )
481
+ return cfl_PyInt_FromInt (self -> leader_epoch );
482
+ else
483
+ Py_RETURN_NONE ;
484
+ }
485
+
479
486
480
487
static PyObject * Message_timestamp (Message * self , PyObject * ignore ) {
481
488
return Py_BuildValue ("iL" ,
@@ -571,6 +578,11 @@ static PyMethodDef Message_methods[] = {
571
578
" :rtype: int or None\n"
572
579
"\n"
573
580
},
581
+ { "leader_epoch" , (PyCFunction )Message_leader_epoch , METH_NOARGS ,
582
+ " :returns: message offset leader epoch or None if not available.\n"
583
+ " :rtype: int or None\n"
584
+ "\n"
585
+ },
574
586
{ "timestamp" , (PyCFunction )Message_timestamp , METH_NOARGS ,
575
587
"Retrieve timestamp type and timestamp from message.\n"
576
588
"The timestamp type is one of:\n\n"
@@ -743,7 +755,7 @@ PyTypeObject MessageType = {
743
755
0 , /* tp_weaklistoffset */
744
756
0 , /* tp_iter */
745
757
0 , /* tp_iternext */
746
- Message_methods , /* tp_methods */
758
+ Message_methods , /* tp_methods */
747
759
0 , /* tp_members */
748
760
0 , /* tp_getset */
749
761
0 , /* tp_base */
@@ -784,6 +796,7 @@ PyObject *Message_new0 (const Handle *handle, const rd_kafka_message_t *rkm) {
784
796
785
797
self -> partition = rkm -> partition ;
786
798
self -> offset = rkm -> offset ;
799
+ self -> leader_epoch = rd_kafka_message_leader_epoch (rkm );
787
800
788
801
self -> timestamp = rd_kafka_message_timestamp (rkm , & self -> tstype );
789
802
@@ -825,12 +838,17 @@ static int TopicPartition_clear (TopicPartition *self) {
825
838
826
839
static void TopicPartition_setup (TopicPartition * self , const char * topic ,
827
840
int partition , long long offset ,
841
+ int32_t leader_epoch ,
828
842
const char * metadata ,
829
843
rd_kafka_resp_err_t err ) {
830
844
self -> topic = strdup (topic );
831
845
self -> partition = partition ;
832
846
self -> offset = offset ;
833
847
848
+ if (leader_epoch < 0 )
849
+ leader_epoch = -1 ;
850
+ self -> leader_epoch = leader_epoch ;
851
+
834
852
if (metadata != NULL ) {
835
853
self -> metadata = strdup (metadata );
836
854
} else {
@@ -854,23 +872,27 @@ static int TopicPartition_init (PyObject *self, PyObject *args,
854
872
PyObject * kwargs ) {
855
873
const char * topic ;
856
874
int partition = RD_KAFKA_PARTITION_UA ;
875
+ int32_t leader_epoch = -1 ;
857
876
long long offset = RD_KAFKA_OFFSET_INVALID ;
858
877
const char * metadata = NULL ;
859
878
860
879
static char * kws [] = { "topic" ,
861
880
"partition" ,
862
881
"offset" ,
863
882
"metadata" ,
883
+ "leader_epoch" ,
864
884
NULL };
865
885
866
- if (!PyArg_ParseTupleAndKeywords (args , kwargs , "s|iLs " , kws ,
886
+ if (!PyArg_ParseTupleAndKeywords (args , kwargs , "s|iLsi " , kws ,
867
887
& topic , & partition , & offset ,
868
- & metadata )) {
888
+ & metadata ,
889
+ & leader_epoch )) {
869
890
return -1 ;
870
891
}
871
892
872
893
TopicPartition_setup ((TopicPartition * )self ,
873
- topic , partition , offset , metadata , 0 );
894
+ topic , partition , offset ,
895
+ leader_epoch , metadata , 0 );
874
896
return 0 ;
875
897
}
876
898
@@ -890,6 +912,13 @@ static int TopicPartition_traverse (TopicPartition *self,
890
912
return 0 ;
891
913
}
892
914
915
+ static PyObject * TopicPartition_get_leader_epoch (TopicPartition * tp , void * closure ) {
916
+ if (tp -> leader_epoch >= 0 ) {
917
+ return cfl_PyInt_FromInt (tp -> leader_epoch );
918
+ }
919
+ Py_RETURN_NONE ;
920
+ }
921
+
893
922
894
923
static PyMemberDef TopicPartition_members [] = {
895
924
{ "topic" , T_STRING , offsetof(TopicPartition , topic ), READONLY ,
@@ -913,15 +942,36 @@ static PyMemberDef TopicPartition_members[] = {
913
942
{ NULL }
914
943
};
915
944
945
+ static PyGetSetDef TopicPartition_getters_and_setters [] = {
946
+ {
947
+ /* name */
948
+ "leader_epoch" ,
949
+ (getter ) TopicPartition_get_leader_epoch ,
950
+ NULL ,
951
+ /* doc */
952
+ ":attribute leader_epoch: Offset leader epoch (int), or None" ,
953
+ /* closure */
954
+ NULL
955
+ },
956
+ { NULL }
957
+ };
958
+
916
959
917
960
static PyObject * TopicPartition_str0 (TopicPartition * self ) {
918
961
PyObject * errstr = NULL ;
919
962
PyObject * errstr8 = NULL ;
920
963
const char * c_errstr = NULL ;
921
964
PyObject * ret ;
922
965
char offset_str [40 ];
966
+ char leader_epoch_str [12 ];
923
967
924
968
snprintf (offset_str , sizeof (offset_str ), "%" CFL_PRId64 "" , self -> offset );
969
+ if (self -> leader_epoch >= 0 )
970
+ snprintf (leader_epoch_str , sizeof (leader_epoch_str ),
971
+ "%" CFL_PRId32 "" , self -> leader_epoch );
972
+ else
973
+ snprintf (leader_epoch_str , sizeof (leader_epoch_str ),
974
+ "None" );
925
975
926
976
if (self -> error != Py_None ) {
927
977
errstr = cfl_PyObject_Unistr (self -> error );
@@ -930,9 +980,10 @@ static PyObject *TopicPartition_str0 (TopicPartition *self) {
930
980
931
981
ret = cfl_PyUnistr (
932
982
_FromFormat ("TopicPartition{topic=%s,partition=%" CFL_PRId32
933
- ",offset=%s,error=%s}" ,
983
+ ",offset=%s,leader_epoch=%s, error=%s}" ,
934
984
self -> topic , self -> partition ,
935
985
offset_str ,
986
+ leader_epoch_str ,
936
987
c_errstr ? c_errstr : "None" ));
937
988
Py_XDECREF (errstr8 );
938
989
Py_XDECREF (errstr );
@@ -1024,48 +1075,53 @@ PyTypeObject TopicPartitionType = {
1024
1075
"It is typically used to provide a list of topics or partitions for "
1025
1076
"various operations, such as :py:func:`Consumer.assign()`.\n"
1026
1077
"\n"
1027
- ".. py:function:: TopicPartition(topic, [partition], [offset])\n"
1078
+ ".. py:function:: TopicPartition(topic, [partition], [offset],"
1079
+ " [metadata], [leader_epoch])\n"
1028
1080
"\n"
1029
1081
" Instantiate a TopicPartition object.\n"
1030
1082
"\n"
1031
1083
" :param string topic: Topic name\n"
1032
1084
" :param int partition: Partition id\n"
1033
1085
" :param int offset: Initial partition offset\n"
1086
+ " :param string metadata: Offset metadata\n"
1087
+ " :param int leader_epoch: Offset leader epoch\n"
1034
1088
" :rtype: TopicPartition\n"
1035
1089
"\n"
1036
1090
"\n" , /*tp_doc*/
1037
1091
(traverseproc )TopicPartition_traverse , /* tp_traverse */
1038
1092
(inquiry )TopicPartition_clear , /* tp_clear */
1039
1093
(richcmpfunc )TopicPartition_richcompare , /* tp_richcompare */
1040
- 0 , /* tp_weaklistoffset */
1041
- 0 , /* tp_iter */
1042
- 0 , /* tp_iternext */
1043
- 0 , /* tp_methods */
1044
- TopicPartition_members ,/* tp_members */
1045
- 0 , /* tp_getset */
1046
- 0 , /* tp_base */
1047
- 0 , /* tp_dict */
1048
- 0 , /* tp_descr_get */
1049
- 0 , /* tp_descr_set */
1050
- 0 , /* tp_dictoffset */
1051
- TopicPartition_init , /* tp_init */
1052
- 0 , /* tp_alloc */
1053
- TopicPartition_new /* tp_new */
1094
+ 0 , /* tp_weaklistoffset */
1095
+ 0 , /* tp_iter */
1096
+ 0 , /* tp_iternext */
1097
+ 0 , /* tp_methods */
1098
+ TopicPartition_members , /* tp_members */
1099
+ TopicPartition_getters_and_setters , /* tp_getset */
1100
+ 0 , /* tp_base */
1101
+ 0 , /* tp_dict */
1102
+ 0 , /* tp_descr_get */
1103
+ 0 , /* tp_descr_set */
1104
+ 0 , /* tp_dictoffset */
1105
+ TopicPartition_init , /* tp_init */
1106
+ 0 , /* tp_alloc */
1107
+ TopicPartition_new /* tp_new */
1054
1108
};
1055
1109
1056
1110
/**
1057
1111
* @brief Internal factory to create a TopicPartition object.
1058
1112
*/
1059
1113
static PyObject * TopicPartition_new0 (const char * topic , int partition ,
1060
- long long offset , const char * metadata ,
1114
+ long long offset , int32_t leader_epoch ,
1115
+ const char * metadata ,
1061
1116
rd_kafka_resp_err_t err ) {
1062
1117
TopicPartition * self ;
1063
1118
1064
1119
self = (TopicPartition * )TopicPartitionType .tp_new (
1065
1120
& TopicPartitionType , NULL , NULL );
1066
1121
1067
1122
TopicPartition_setup (self , topic , partition ,
1068
- offset , metadata , err );
1123
+ offset , leader_epoch ,
1124
+ metadata , err );
1069
1125
1070
1126
return (PyObject * )self ;
1071
1127
}
@@ -1090,6 +1146,7 @@ PyObject *c_parts_to_py (const rd_kafka_topic_partition_list_t *c_parts) {
1090
1146
TopicPartition_new0 (
1091
1147
rktpar -> topic , rktpar -> partition ,
1092
1148
rktpar -> offset ,
1149
+ rd_kafka_topic_partition_get_leader_epoch (rktpar ),
1093
1150
rktpar -> metadata ,
1094
1151
rktpar -> err ));
1095
1152
}
@@ -1133,6 +1190,8 @@ rd_kafka_topic_partition_list_t *py_to_c_parts (PyObject *plist) {
1133
1190
tp -> topic ,
1134
1191
tp -> partition );
1135
1192
rktpar -> offset = tp -> offset ;
1193
+ rd_kafka_topic_partition_set_leader_epoch (rktpar ,
1194
+ tp -> leader_epoch );
1136
1195
if (tp -> metadata != NULL ) {
1137
1196
rktpar -> metadata_size = strlen (tp -> metadata ) + 1 ;
1138
1197
rktpar -> metadata = strdup (tp -> metadata );
0 commit comments