Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KIP 951 - Phase 1 #4613

Merged
merged 10 commits into from
Jun 14, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
47 changes: 47 additions & 0 deletions src/rdkafka_buf.h
Original file line number Diff line number Diff line change
Expand Up @@ -836,6 +836,53 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */
} \
} while (0)

/**
* @brief Read KIP-482 Tags at current position in the buffer using
* the `read_tag` function receiving the `opaque' pointer.
*/
#define rd_kafka_buf_read_tags(rkbuf, read_tag, ...) \
do { \
uint64_t _tagcnt; \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagcnt); \
while (_tagcnt-- > 0) { \
uint64_t _tagtype, _taglen; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagtype); \
rd_kafka_buf_read_uvarint(rkbuf, &_taglen); \
int _read_tag_resp = \
read_tag(rkbuf, _tagtype, _taglen, __VA_ARGS__); \
if (_read_tag_resp == -1) \
goto err_parse; \
if (!_read_tag_resp && _taglen > 0) \
rd_kafka_buf_skip(rkbuf, (size_t)(_taglen)); \
} \
} while (0)

anchitj marked this conversation as resolved.
Show resolved Hide resolved
/**
* @brief Write \p tagcnt tags at the current position in the buffer.
* Calling \p write_tag to write each one with \p rkbuf , tagtype
* argument and the remaining arguments.
*/
#define rd_kafka_buf_write_tags(rkbuf, write_tag, tags, tagcnt, ...) \
do { \
uint64_t i; \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_write_uvarint(rkbuf, tagcnt); \
for (i = 0; i < tagcnt; i++) { \
size_t of_taglen, prev_buf_len; \
rd_kafka_buf_write_uvarint(rkbuf, tags[i]); \
of_taglen = rd_kafka_buf_write_arraycnt_pos(rkbuf); \
prev_buf_len = (rkbuf)->rkbuf_buf.rbuf_len; \
write_tag(rkbuf, tags[i], __VA_ARGS__); \
rd_kafka_buf_finalize_arraycnt( \
rkbuf, of_taglen, \
(rkbuf)->rkbuf_buf.rbuf_len - prev_buf_len - 1); \
} \
} while (0)


/**
* @brief Write empty tags at the current position in the buffer.
*/
Expand Down
55 changes: 55 additions & 0 deletions src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -497,6 +497,61 @@ int rd_kafka_buf_write_topic_partitions(
}


/**
* @brief Read current leader from \p rkbuf.
*
* @param rkbuf buffer to read from
* @param CurrentLeader is the CurrentLeader to populate.
*
* @return 1 on success, else -1 on parse error.
anchitj marked this conversation as resolved.
Show resolved Hide resolved
*/
int rd_kafka_buf_read_CurrentLeader(rd_kafka_buf_t *rkbuf,
rd_kafkap_CurrentLeader_t *CurrentLeader) {
const int log_decode_errors = LOG_ERR;
rd_kafka_buf_read_i32(rkbuf, &CurrentLeader->LeaderId);
rd_kafka_buf_read_i32(rkbuf, &CurrentLeader->LeaderEpoch);
rd_kafka_buf_skip_tags(rkbuf);
return 1;
err_parse:
return -1;
}

/**
* @brief Read NodeEndpoints from \p rkbuf.
*
* @param rkbuf buffer to read from
* @param NodeEndpoints is the NodeEndpoints to populate.
*
* @return 1 on success, else -1 on parse error.
*/
int rd_kafka_buf_read_NodeEndpoints(rd_kafka_buf_t *rkbuf,
rd_kafkap_NodeEndpoints_t *NodeEndpoints) {
const int log_decode_errors = LOG_ERR;
int32_t i;
rd_kafka_buf_read_arraycnt(rkbuf, &NodeEndpoints->NodeEndpointCnt,
RD_KAFKAP_BROKERS_MAX);
rd_dassert(!NodeEndpoints->NodeEndpoints);
NodeEndpoints->NodeEndpoints =
rd_calloc(NodeEndpoints->NodeEndpointCnt,
sizeof(*NodeEndpoints->NodeEndpoints));

for (i = 0; i < NodeEndpoints->NodeEndpointCnt; i++) {
rd_kafka_buf_read_i32(rkbuf,
&NodeEndpoints->NodeEndpoints[i].NodeId);
rd_kafka_buf_read_str(rkbuf,
&NodeEndpoints->NodeEndpoints[i].Host);
rd_kafka_buf_read_i32(rkbuf,
&NodeEndpoints->NodeEndpoints[i].Port);
rd_kafka_buf_read_str(rkbuf,
&NodeEndpoints->NodeEndpoints[i].Rack);
rd_kafka_buf_skip_tags(rkbuf);
}
return 1;
err_parse:
return -1;
}


/**
* @brief Send FindCoordinatorRequest.
*
Expand Down
33 changes: 33 additions & 0 deletions src/rdkafka_request.h
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,32 @@ typedef enum {
RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP,
} rd_kafka_topic_partition_field_t;

/**
* @name Current Leader and NodeEndpoints for KIP-951
emasab marked this conversation as resolved.
Show resolved Hide resolved
* response triggered metadata updates.
*
* @{
*/

typedef struct rd_kafkap_CurrentLeader_s {
int32_t LeaderId;
int32_t LeaderEpoch;
} rd_kafkap_CurrentLeader_t;

typedef struct rd_kafkap_NodeEndpoint_s {
int32_t NodeId;
rd_kafkap_str_t Host;
int32_t Port;
rd_kafkap_str_t Rack;
} rd_kafkap_NodeEndpoint_t;

typedef struct rd_kafkap_NodeEndpoints_s {
int32_t NodeEndpointCnt;
rd_kafkap_NodeEndpoint_t *NodeEndpoints;
} rd_kafkap_NodeEndpoints_t;

/**@}*/

rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
rd_kafka_buf_t *rkbuf,
rd_bool_t use_topic_id,
Expand All @@ -99,6 +125,13 @@ int rd_kafka_buf_write_topic_partitions(
rd_bool_t use_topic_name,
const rd_kafka_topic_partition_field_t *fields);

int rd_kafka_buf_read_CurrentLeader(rd_kafka_buf_t *rkbuf,
rd_kafkap_CurrentLeader_t *CurrentLeader);

int rd_kafka_buf_read_NodeEndpoints(rd_kafka_buf_t *rkbuf,
rd_kafkap_NodeEndpoints_t *NodeEndpoints);


rd_kafka_resp_err_t
rd_kafka_FindCoordinatorRequest(rd_kafka_broker_t *rkb,
rd_kafka_coordtype_t coordtype,
Expand Down