Skip to content
Draft
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
160 changes: 160 additions & 0 deletions src/rdkafka.h
Original file line number Diff line number Diff line change
Expand Up @@ -5610,6 +5610,8 @@ typedef int rd_kafka_event_type_t;
#define RD_KAFKA_EVENT_LISTOFFSETS_RESULT 0x400000
/** ElectLeaders_result_t */
#define RD_KAFKA_EVENT_ELECTLEADERS_RESULT 0x800000
/** DescribeLogDirs_result_t */
#define RD_KAFKA_EVENT_DESCRIBELOGDIRS_RESULT 0x1000000

/**
* @returns the event type for the given event.
Expand Down Expand Up @@ -5894,6 +5896,8 @@ typedef rd_kafka_event_t rd_kafka_AlterUserScramCredentials_result_t;
typedef rd_kafka_event_t rd_kafka_ListOffsets_result_t;
/*! ElectLeaders result type */
typedef rd_kafka_event_t rd_kafka_ElectLeaders_result_t;
/*! DescribeLogDirs result type */
typedef rd_kafka_event_t rd_kafka_DescribeLogDirs_result_t;

/**
* @brief Get CreateTopics result.
Expand Down Expand Up @@ -7118,6 +7122,7 @@ typedef enum rd_kafka_admin_op_t {
RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, /**< DescribeCluster */
RD_KAFKA_ADMIN_OP_LISTOFFSETS, /**< ListOffsets */
RD_KAFKA_ADMIN_OP_ELECTLEADERS, /**< ElectLeaders */
RD_KAFKA_ADMIN_OP_DESCRIBELOGDIRS, /**< DescribeLogDirs */
RD_KAFKA_ADMIN_OP__CNT /**< Number of ops defined */
} rd_kafka_admin_op_t;

Expand Down Expand Up @@ -10140,6 +10145,161 @@ rd_kafka_ElectLeaders_result_partitions(

/**@}*/

/**
* @name Admin API - Describe Log Dirs
* @{
*
*
*
*/

/**
* @brief Opaque type for a log directory description.
*/
typedef struct rd_kafka_LogDirDescription_s rd_kafka_LogDirDescription_t;

/**
* @brief Opaque type for a log directory topic description.
*/
typedef struct rd_kafka_LogDirTopicDescription_s
rd_kafka_LogDirTopicDescription_t;

/**
* @brief Opaque type for a log directory partition description.
*/
typedef struct rd_kafka_LogDirPartitionDescription_s
rd_kafka_LogDirPartitionDescription_t;

/**
* @brief Describe log directories on a broker.
*
* @param rk Client instance.
* @param topics Topic-partition list to filter results, or NULL to describe
* all topics on the target broker.
* @param options Optional admin options, or NULL for defaults.
* Use rd_kafka_AdminOptions_set_broker() to target a specific broker.
* Defaults to controller.
* @param rkqu Queue to emit result on.
*
* Supported admin options:
* - rd_kafka_AdminOptions_set_request_timeout() - default socket.timeout.ms.
* Controls how long \c rdkafka will wait for the request to complete.
* - rd_kafka_AdminOptions_set_broker() - target a specific broker.
*
* @remark The result event type emitted on the supplied queue is of type
* \c RD_KAFKA_EVENT_DESCRIBELOGDIRS_RESULT
*/
RD_EXPORT void rd_kafka_DescribeLogDirs(
rd_kafka_t *rk,
const rd_kafka_topic_partition_list_t *topics,
const rd_kafka_AdminOptions_t *options,
rd_kafka_queue_t *rkqu);

/**
* @brief Get the array of log dir description objects from the
* DescribeLogDirs result event.
*
* @param result The DescribeLogDirs result.
* @param cntp Is updated to the number of elements in the array.
*
* @returns the array of log dir descriptions.
*/
RD_EXPORT const rd_kafka_LogDirDescription_t **
rd_kafka_DescribeLogDirs_result_descriptions(
const rd_kafka_DescribeLogDirs_result_t *result,
size_t *cntp);

/**
* @brief Get the DescribeLogDirs result from an event.
*
* @param rkev Event of type \c RD_KAFKA_EVENT_DESCRIBELOGDIRS_RESULT.
*
* @returns the result, or NULL if event is of different type.
*/
RD_EXPORT const rd_kafka_DescribeLogDirs_result_t *
rd_kafka_event_DescribeLogDirs_result(rd_kafka_event_t *rkev);

/**
* @brief Get the error code from a log dir description.
*/
RD_EXPORT rd_kafka_resp_err_t
rd_kafka_LogDirDescription_error_code(
const rd_kafka_LogDirDescription_t *logdir);

/**
* @brief Get the log directory path from a log dir description.
*/
RD_EXPORT const char *
rd_kafka_LogDirDescription_log_dir(
const rd_kafka_LogDirDescription_t *logdir);

/**
* @brief Get the array of topic descriptions from a log dir description.
*
* @param logdir The log dir description.
* @param cntp Is updated to the number of topics.
*
* @returns the array of topic descriptions.
*/
RD_EXPORT const rd_kafka_LogDirTopicDescription_t **
rd_kafka_LogDirDescription_topics(
const rd_kafka_LogDirDescription_t *logdir,
size_t *cntp);

/**
* @brief Get the topic name from a log dir topic description.
*/
RD_EXPORT const char *
rd_kafka_LogDirTopicDescription_topic(
const rd_kafka_LogDirTopicDescription_t *topic_desc);

/**
* @brief Get the array of partition descriptions from a log dir topic
* description.
*
* @param topic_desc The topic description.
* @param cntp Is updated to the number of partitions.
*
* @returns the array of partition descriptions.
*/
RD_EXPORT const rd_kafka_LogDirPartitionDescription_t **
rd_kafka_LogDirTopicDescription_partitions(
const rd_kafka_LogDirTopicDescription_t *topic_desc,
size_t *cntp);

/**
* @brief Get the partition index from a log dir partition description.
*/
RD_EXPORT int32_t
rd_kafka_LogDirPartitionDescription_partition(
const rd_kafka_LogDirPartitionDescription_t *partition_desc);

/**
* @brief Get the partition size in bytes from a log dir partition description.
*/
RD_EXPORT int64_t
rd_kafka_LogDirPartitionDescription_size(
const rd_kafka_LogDirPartitionDescription_t *partition_desc);

/**
* @brief Get the offset lag from a log dir partition description.
*/
RD_EXPORT int64_t
rd_kafka_LogDirPartitionDescription_offset_lag(
const rd_kafka_LogDirPartitionDescription_t *partition_desc);

/**
* @brief Get the is_future flag from a log dir partition description.
*
* @returns 1 if this is a future log directory (i.e., the partition is being
* moved to this log dir), 0 otherwise.
*/
RD_EXPORT int
rd_kafka_LogDirPartitionDescription_is_future(
const rd_kafka_LogDirPartitionDescription_t *partition_desc);

/**@}*/

/**
* @name Security APIs
* @{
Expand Down
Loading