@@ -2911,8 +2911,19 @@ const char *rd_kafka_ResourcePatternType_name(
29112911}
29122912
29132913const  char  * rd_kafka_ResourceType_name (rd_kafka_ResourceType_t  restype ) {
2914-         static  const  char  * names [] =  {"UNKNOWN" , "ANY" ,    "TOPIC" ,
2915-                                       "GROUP" ,   "BROKER" , "TRANSACTIONAL_ID" };
2914+         static  const  char  * names [] =  {"UNKNOWN" ,
2915+                                       "ANY" ,
2916+                                       "TOPIC" ,
2917+                                       "GROUP" ,
2918+                                       "BROKER" ,
2919+                                       "TRANSACTIONAL_ID" ,
2920+                                       "DELEGATION_TOKEN" ,
2921+                                       "USER" };
2922+ 
2923+         if  ((unsigned int  )restype  >= RD_KAFKA_RESOURCE_CONFIG_RESOURCE__START  && 
2924+             (unsigned int  )restype  <  RD_KAFKA_RESOURCE_CONFIG_RESOURCE__END )
2925+                 return  rd_kafka_ConfigResourceType_name (
2926+                     rd_kafka_ResourceType_to_ConfigResourceType (restype ));
29162927
29172928        if  ((unsigned int  )restype  >= (unsigned int  )RD_KAFKA_RESOURCE__CNT )
29182929                return  "UNSUPPORTED" ;
@@ -2928,6 +2939,10 @@ rd_kafka_ResourceType_to_ConfigResourceType(rd_kafka_ResourceType_t restype) {
29282939                return  RD_KAFKA_CONFIG_RESOURCE_TOPIC ;
29292940        case  RD_KAFKA_RESOURCE_BROKER :
29302941                return  RD_KAFKA_CONFIG_RESOURCE_BROKER ;
2942+         case  RD_KAFKA_RESOURCE_BROKER_LOGGER :
2943+                 return  RD_KAFKA_CONFIG_RESOURCE_BROKER_LOGGER ;
2944+         case  RD_KAFKA_RESOURCE_CLIENT_METRICS :
2945+                 return  RD_KAFKA_CONFIG_RESOURCE_CLIENT_METRICS ;
29312946        case  RD_KAFKA_RESOURCE_GROUP :
29322947                return  RD_KAFKA_CONFIG_RESOURCE_GROUP ;
29332948        default :
@@ -2942,12 +2957,33 @@ rd_kafka_ResourceType_t rd_kafka_ConfigResourceType_to_ResourceType(
29422957                return  RD_KAFKA_RESOURCE_TOPIC ;
29432958        case  RD_KAFKA_CONFIG_RESOURCE_BROKER :
29442959                return  RD_KAFKA_RESOURCE_BROKER ;
2960+         case  RD_KAFKA_CONFIG_RESOURCE_BROKER_LOGGER :
2961+                 return  RD_KAFKA_RESOURCE_BROKER_LOGGER ;
2962+         case  RD_KAFKA_CONFIG_RESOURCE_CLIENT_METRICS :
2963+                 return  RD_KAFKA_RESOURCE_CLIENT_METRICS ;
29452964        case  RD_KAFKA_CONFIG_RESOURCE_GROUP :
29462965                return  RD_KAFKA_RESOURCE_GROUP ;
29472966        default :
29482967                return  RD_KAFKA_RESOURCE_UNKNOWN ;
29492968        }
29502969}
2970+ /** 
2971+  * @brief Get the name of a ConfigResourceType_t. 
2972+  */ 
2973+ const  char  * rd_kafka_ConfigResourceType_name (
2974+     rd_kafka_ConfigResourceType_t  config_resource_type ) {
2975+         static  const  char  * names [] =  {
2976+             [0 ] =  "UNKNOWN" ,       [2 ] =  "TOPIC" ,           [4 ] =  "BROKER" ,
2977+             [8 ] =  "BROKER_LOGGER" , [16 ] =  "CLIENT_METRICS" , [32 ] =  "GROUP" };
2978+         if  ((unsigned int  )config_resource_type  >=
2979+             (unsigned int  )RD_KAFKA_CONFIG_RESOURCE__END )
2980+                 return  "UNSUPPORTED" ;
2981+ 
2982+         if  (!names [(unsigned int  )config_resource_type ])
2983+                 return  "UNSUPPORTED" ;
2984+ 
2985+         return  names [config_resource_type ];
2986+ }
29512987
29522988
29532989rd_kafka_ConfigResource_t  * 
0 commit comments