Skip to content

Commit

Permalink
Some Changes
Browse files Browse the repository at this point in the history
  • Loading branch information
mahajanadhitya committed Apr 3, 2023
1 parent 09b25ab commit c789847
Show file tree
Hide file tree
Showing 4 changed files with 21 additions and 4 deletions.
7 changes: 7 additions & 0 deletions src/rdkafka.h
Original file line number Diff line number Diff line change
Expand Up @@ -5350,6 +5350,8 @@ typedef int rd_kafka_event_type_t;
#define RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT 0x10000
/* Describe User Scram Credentials*/
#define RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT 0x20000
/* Alter User Scram Credentials*/
#define RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT 0x40000


/**
Expand Down Expand Up @@ -5618,6 +5620,8 @@ typedef rd_kafka_event_t rd_kafka_AlterConsumerGroupOffsets_result_t;
typedef rd_kafka_event_t rd_kafka_ListConsumerGroupOffsets_result_t;
/*! DescribeUserScramCredentials result type */
typedef rd_kafka_event_t rd_kafka_DescribeUserScramCredentials_result_t;
/*! AlterUserScramCredentials result type */
typedef rd_kafka_event_t rd_kafka_AlterUserScramCredentials_result_t;

/**
* @brief Get CreateTopics result.
Expand Down Expand Up @@ -5791,6 +5795,9 @@ rd_kafka_event_AlterConsumerGroupOffsets_result(rd_kafka_event_t *rkev);

RD_EXPORT const rd_kafka_DescribeUserScramCredentials_result_t *
rd_kafka_event_DescribeUserScramCredentials_result(rd_kafka_event_t *rkev);

RD_EXPORT const rd_kafka_AlterUserScramCredentials_result_t *
rd_kafka_event_AlterUserScramCredentials_result(rd_kafka_event_t *rkev);
/**
* @brief Get ListConsumerGroupOffsets result.
*
Expand Down
6 changes: 3 additions & 3 deletions src/rdkafka_admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -4974,7 +4974,7 @@ rd_kafka_resp_err_t rd_kafka_DescribeUserScramCredentialsRequest(rd_kafka_broker
void *opaque){

rd_kafka_buf_t *rkbuf;
int16_t ApiVersion = 0;
int16_t ApiVersion = 0;
int num_users = rd_list_cnt(userlist);

rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_DescribeUserScramCredentials, 1,
Expand Down Expand Up @@ -5070,7 +5070,7 @@ rd_kafka_DescribeUserScramCredentialsResponse_parse(rd_kafka_op_t *rko_req,
int32_t iterations;
rd_kafka_buf_read_i8(reply,&mechanism);
rd_kafka_buf_read_i32(reply,&iterations);
rd_kafka_scram_credential_t *scram_credential = rd_kafka_scram_credential_new(mechanism,iterations);
rd_kafka_scram_credential_t *scram_credential = rd_kafka_scram_credential_new(&username->str,mechanism,iterations);
scram_credential->user = rd_strdup(&username->str);
scram_credential->errorcode = user_error_code;
scram_credential->err = rd_strdup(&user_error_msg->str);
Expand Down Expand Up @@ -5124,7 +5124,7 @@ rd_kafka_resp_err_t rd_kafka_AlterUserScramCredentialsRequest(rd_kafka_broker_t
int16_t ApiVersion = 0;
int num_credentials = rd_list_cnt(scram_credentials);

rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_DescribeUserScramCredentials, 1,
rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_AlterUserScramCredentials, 1,
num_credentials*100);
size_t of_deletions = rd_kafka_buf_write_arraycnt_pos(rkbuf);

Expand Down
11 changes: 10 additions & 1 deletion src/rdkafka_event.c
Original file line number Diff line number Diff line change
Expand Up @@ -425,9 +425,18 @@ rd_kafka_event_DescribeUserScramCredentials_result(rd_kafka_event_t *rkev){
return NULL;
else
return (
const rd_kafka_AlterConsumerGroupOffsets_result_t *)rkev;
const rd_kafka_DescribeUserScramCredentials_result_t *)rkev;
}

const rd_kafka_AlterUserScramCredentials_result_t *
rd_kafka_event_AlterUserScramCredentials_result(rd_kafka_event_t *rkev){
if (!rkev ||
rkev->rko_evtype != RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT)
return NULL;
else
return (
const rd_kafka_AlterUserScramCredentials_result_t *)rkev;
}
const rd_kafka_ListConsumerGroupOffsets_result_t *
rd_kafka_event_ListConsumerGroupOffsets_result(rd_kafka_event_t *rkev) {
if (!rkev ||
Expand Down
1 change: 1 addition & 0 deletions src/rdkafka_op.h
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,7 @@ typedef enum {
RD_KAFKA_OP_LEADERS, /**< Partition leader query */
RD_KAFKA_OP_BARRIER, /**< Version barrier bump */
RD_KAFKA_OP_DESCRIBEUSERSCRAMCREDENTIALS, /* Describe user scram credential KIP*/
RD_KAFKA_OP_ALTERUSERSCRAMCREDENTIALS, /* Alter user scram credential KIP*/
RD_KAFKA_OP__END
} rd_kafka_op_type_t;

Expand Down

0 comments on commit c789847

Please sign in to comment.