@@ -333,6 +333,34 @@ static PyObject *Message_timestamp (Message *self, PyObject *ignore) {
333333 self -> timestamp );
334334}
335335
336+ static PyObject * Message_headers (Message * self , PyObject * ignore ) {
337+ #ifdef RD_KAFKA_V_HEADERS
338+ if (self -> headers ) {
339+ Py_INCREF (self -> headers );
340+ return self -> headers ;
341+ } else if (self -> c_headers ) {
342+ self -> headers = c_headers_to_py (self -> c_headers );
343+ rd_kafka_headers_destroy (self -> c_headers );
344+ self -> c_headers = NULL ;
345+ Py_INCREF (self -> headers );
346+ return self -> headers ;
347+ } else {
348+ Py_RETURN_NONE ;
349+ }
350+ #else
351+ Py_RETURN_NONE ;
352+ #endif
353+ }
354+
355+ static PyObject * Message_set_headers (Message * self , PyObject * new_headers ) {
356+ if (self -> headers )
357+ Py_DECREF (self -> headers );
358+ self -> headers = new_headers ;
359+ Py_INCREF (self -> headers );
360+
361+ Py_RETURN_NONE ;
362+ }
363+
336364static PyObject * Message_set_value (Message * self , PyObject * new_val ) {
337365 if (self -> value )
338366 Py_DECREF (self -> value );
@@ -409,6 +437,21 @@ static PyMethodDef Message_methods[] = {
409437 " :rtype: (int, int)\n"
410438 "\n"
411439 },
440+ { "headers" , (PyCFunction )Message_headers , METH_NOARGS ,
441+ " Retrieve the headers set on a message. Each header is a key value"
442+ "pair. Please note that header keys are ordered and can repeat.\n"
443+ "\n"
444+ " :returns: list of two-tuples, one (key, value) pair for each header.\n"
445+ " :rtype: [(str, bytes),...] or None.\n"
446+ "\n"
447+ },
448+ { "set_headers" , (PyCFunction )Message_set_headers , METH_O ,
449+ " Set the field 'Message.headers' with new value.\n"
450+ " :param: object value: Message.headers.\n"
451+ " :returns: None.\n"
452+ " :rtype: None\n"
453+ "\n"
454+ },
412455 { "set_value" , (PyCFunction )Message_set_value , METH_O ,
413456 " Set the field 'Message.value' with new value.\n"
414457 " :param: object value: Message.value.\n"
@@ -443,6 +486,16 @@ static int Message_clear (Message *self) {
443486 Py_DECREF (self -> error );
444487 self -> error = NULL ;
445488 }
489+ if (self -> headers ) {
490+ Py_DECREF (self -> headers );
491+ self -> headers = NULL ;
492+ }
493+ #ifdef RD_KAFKA_V_HEADERS
494+ if (self -> c_headers ){
495+ rd_kafka_headers_destroy (self -> c_headers );
496+ self -> c_headers = NULL ;
497+ }
498+ #endif
446499 return 0 ;
447500}
448501
@@ -464,6 +517,8 @@ static int Message_traverse (Message *self,
464517 Py_VISIT (self -> key );
465518 if (self -> error )
466519 Py_VISIT (self -> error );
520+ if (self -> headers )
521+ Py_VISIT (self -> headers );
467522 return 0 ;
468523}
469524
@@ -883,6 +938,81 @@ rd_kafka_topic_partition_list_t *py_to_c_parts (PyObject *plist) {
883938 return c_parts ;
884939}
885940
941+ #ifdef RD_KAFKA_V_HEADERS
942+ /**
943+ * @brief Convert Python list[(header_key, header_value),...]) to C rd_kafka_topic_partition_list_t.
944+ *
945+ * @returns The new Python list[(header_key, header_value),...] object.
946+ */
947+ rd_kafka_headers_t * py_headers_to_c (PyObject * headers_plist ) {
948+ int i , len ;
949+ rd_kafka_headers_t * rd_headers = NULL ;
950+ rd_kafka_resp_err_t err ;
951+ const char * header_key , * header_value = NULL ;
952+ int header_key_len = 0 , header_value_len = 0 ;
953+
954+ len = PyList_Size (headers_plist );
955+ rd_headers = rd_kafka_headers_new (len );
956+
957+ for (i = 0 ; i < len ; i ++ ) {
958+
959+ if (!PyArg_ParseTuple (PyList_GET_ITEM (headers_plist , i ), "s#z#" , & header_key ,
960+ & header_key_len , & header_value , & header_value_len )){
961+ rd_kafka_headers_destroy (rd_headers );
962+ PyErr_SetString (PyExc_TypeError ,
963+ "Headers are expected to be a tuple of (key, value)" );
964+ return NULL ;
965+ }
966+
967+ err = rd_kafka_header_add (rd_headers , header_key , header_key_len , header_value , header_value_len );
968+ if (err ) {
969+ rd_kafka_headers_destroy (rd_headers );
970+ cfl_PyErr_Format (err ,
971+ "Unable to create message headers: %s" ,
972+ rd_kafka_err2str (err ));
973+ return NULL ;
974+ }
975+ }
976+ return rd_headers ;
977+ }
978+
979+ /**
980+ * @brief Convert rd_kafka_headers_t to Python list[(header_key, header_value),...])
981+ *
982+ * @returns The new C headers on success or NULL on error.
983+ */
984+ PyObject * c_headers_to_py (rd_kafka_headers_t * headers ) {
985+ size_t idx = 0 ;
986+ size_t header_size = 0 ;
987+ const char * header_key ;
988+ const void * header_value ;
989+ size_t header_value_size ;
990+ PyObject * header_list ;
991+
992+ header_size = rd_kafka_header_cnt (headers );
993+ header_list = PyList_New (header_size );
994+
995+ while (!rd_kafka_header_get_all (headers , idx ++ ,
996+ & header_key , & header_value , & header_value_size )) {
997+ // Create one (key, value) tuple for each header
998+ PyObject * header_tuple = PyTuple_New (2 );
999+ PyTuple_SetItem (header_tuple , 0 ,
1000+ cfl_PyUnistr (_FromString (header_key ))
1001+ );
1002+
1003+ if (header_value ) {
1004+ PyTuple_SetItem (header_tuple , 1 ,
1005+ cfl_PyBin (_FromStringAndSize (header_value , header_value_size ))
1006+ );
1007+ } else {
1008+ PyTuple_SetItem (header_tuple , 1 , Py_None );
1009+ }
1010+ PyList_SET_ITEM (header_list , idx - 1 , header_tuple );
1011+ }
1012+
1013+ return header_list ;
1014+ }
1015+ #endif
8861016
8871017/****************************************************************************
8881018 *
0 commit comments