librdkafka
The Apache Kafka C/C++ client library
|
Apache Kafka C/C++ consumer and producer client library. More...
Go to the source code of this file.
Data Structures | |
struct | rd_kafka_err_desc |
Error code value, name and description. Typically for use with language bindings to automatically expose the full set of librdkafka error codes. More... | |
struct | rd_kafka_topic_partition_s |
Topic+Partition place holder. More... | |
struct | rd_kafka_topic_partition_list_s |
A growable list of Topic+Partitions. More... | |
struct | rd_kafka_vu_s |
VTYPE + argument container for use with rd_kafka_produce_va() More... | |
struct | rd_kafka_message_s |
A Kafka message as returned by the rd_kafka_consume* () family of functions as well as provided to the Producer dr_msg_cb() . More... | |
struct | rd_kafka_metadata_broker |
Broker information. More... | |
struct | rd_kafka_metadata_partition |
Partition information. More... | |
struct | rd_kafka_metadata_topic |
Topic information. More... | |
struct | rd_kafka_metadata |
Metadata container. More... | |
struct | rd_kafka_group_member_info |
Group member information. More... | |
struct | rd_kafka_group_info |
Group information. More... | |
struct | rd_kafka_group_list |
List of groups. More... | |
Macros | |
#define | RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION |
#define | RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS |
#define | RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE |
#define | RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP RD_KAFKA_RESP_ERR_NOT_COORDINATOR |
Typedefs | |
Plugin interface | |
A plugin interface that allows external runtime-loaded libraries to integrate with a client instance without modifications to the application code. Plugins are loaded when referenced through the
| |
typedef rd_kafka_resp_err_t() | rd_kafka_plugin_f_conf_init_t(rd_kafka_conf_t *conf, void **plug_opaquep, char *errstr, size_t errstr_size) |
Plugin's configuration initializer method called each time the library is referenced from configuration (even if previously loaded by another client instance). More... | |
Functions | |
Message headers | |
Message headers consist of a list of (string key, binary value) pairs. Duplicate keys are supported and the order in which keys were added are retained. Header values are considered binary and may have three types of value:
Headers require Apache Kafka broker version v0.11.0.0 or later. Header operations are O(n). | |
RD_EXPORT rd_kafka_headers_t * | rd_kafka_headers_new (size_t initial_count) |
Create a new headers list. More... | |
RD_EXPORT void | rd_kafka_headers_destroy (rd_kafka_headers_t *hdrs) |
Destroy the headers list. The object and any returned value pointers are not usable after this call. | |
RD_EXPORT rd_kafka_headers_t * | rd_kafka_headers_copy (const rd_kafka_headers_t *src) |
Make a copy of headers list src . | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_header_add (rd_kafka_headers_t *hdrs, const char *name, ssize_t name_size, const void *value, ssize_t value_size) |
Add header with name name and value val (copied) of size size (not including null-terminator). More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_header_remove (rd_kafka_headers_t *hdrs, const char *name) |
Remove all headers for the given key (if any). More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_header_get_last (const rd_kafka_headers_t *hdrs, const char *name, const void **valuep, size_t *sizep) |
Find last header in list hdrs matching name . More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_header_get (const rd_kafka_headers_t *hdrs, size_t idx, const char *name, const void **valuep, size_t *sizep) |
Iterator for headers matching name . More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_header_get_all (const rd_kafka_headers_t *hdrs, size_t idx, const char **namep, const void **valuep, size_t *sizep) |
Iterator for all headers. More... | |
UUID | |
const RD_EXPORT char * | rd_kafka_Uuid_base64str (const rd_kafka_Uuid_t *uuid) |
Computes base64 encoding for the given uuid string. More... | |
RD_EXPORT int64_t | rd_kafka_Uuid_least_significant_bits (const rd_kafka_Uuid_t *uuid) |
Gets least significant 64 bits for the given UUID. More... | |
RD_EXPORT int64_t | rd_kafka_Uuid_most_significant_bits (const rd_kafka_Uuid_t *uuid) |
Gets most significant 64 bits for the given UUID. More... | |
RD_EXPORT rd_kafka_Uuid_t * | rd_kafka_Uuid_new (int64_t most_significant_bits, int64_t least_significant_bits) |
Creates a new UUID. More... | |
RD_EXPORT rd_kafka_Uuid_t * | rd_kafka_Uuid_copy (const rd_kafka_Uuid_t *uuid) |
Copies the given UUID. More... | |
RD_EXPORT void | rd_kafka_Uuid_destroy (rd_kafka_Uuid_t *uuid) |
Destroy the provided uuid. More... | |
Topic configuration | |
Topic configuration property interface | |
RD_EXPORT rd_kafka_topic_conf_t * | rd_kafka_topic_conf_new (void) |
Create topic configuration object. More... | |
RD_EXPORT rd_kafka_topic_conf_t * | rd_kafka_topic_conf_dup (const rd_kafka_topic_conf_t *conf) |
Creates a copy/duplicate of topic configuration object conf . | |
RD_EXPORT rd_kafka_topic_conf_t * | rd_kafka_default_topic_conf_dup (rd_kafka_t *rk) |
Creates a copy/duplicate of rk 's default topic configuration object. | |
RD_EXPORT void | rd_kafka_topic_conf_destroy (rd_kafka_topic_conf_t *topic_conf) |
Destroys a topic conf object. | |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_topic_conf_set (rd_kafka_topic_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size) |
Sets a single rd_kafka_topic_conf_t value by property name. More... | |
RD_EXPORT void | rd_kafka_topic_conf_set_opaque (rd_kafka_topic_conf_t *conf, void *rkt_opaque) |
Sets the application's opaque pointer that will be passed to all topic callbacks as the rkt_opaque argument. More... | |
RD_EXPORT void | rd_kafka_topic_conf_set_partitioner_cb (rd_kafka_topic_conf_t *topic_conf, int32_t(*partitioner)(const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)) |
Producer: Set partitioner callback in provided topic conf object. More... | |
RD_EXPORT void | rd_kafka_topic_conf_set_msg_order_cmp (rd_kafka_topic_conf_t *topic_conf, int(*msg_order_cmp)(const rd_kafka_message_t *a, const rd_kafka_message_t *b)) |
Producer: Set message queueing order comparator callback. More... | |
RD_EXPORT int | rd_kafka_topic_partition_available (const rd_kafka_topic_t *rkt, int32_t partition) |
Check if partition is available (has a leader broker). More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Random partitioner. More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_consistent (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Consistent partitioner. More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_consistent_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Consistent-Random partitioner. More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_murmur2 (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Murmur2 partitioner (Java compatible). More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_murmur2_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Consistent-Random Murmur2 partitioner (Java compatible). More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_fnv1a (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
FNV-1a partitioner. More... | |
RD_EXPORT int32_t | rd_kafka_msg_partitioner_fnv1a_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) |
Consistent-Random FNV-1a partitioner. More... | |
Queue API | |
Message queues allows the application to re-route consumed messages from multiple topic+partitions into one single queue point. This queue point containing messages from a number of topic+partitions may then be served by a single rd_kafka_consume*_queue() call, rather than one call per topic+partition combination. | |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_new (rd_kafka_t *rk) |
Create a new message queue. More... | |
RD_EXPORT void | rd_kafka_queue_destroy (rd_kafka_queue_t *rkqu) |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_get_main (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_get_sasl (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_error_t * | rd_kafka_sasl_background_callbacks_enable (rd_kafka_t *rk) |
Enable SASL OAUTHBEARER refresh callbacks on the librdkafka background thread. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_sasl_set_credentials (rd_kafka_t *rk, const char *username, const char *password) |
Sets SASL credentials used for SASL PLAIN and SCRAM mechanisms by this Kafka client. More... | |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_get_consumer (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_get_partition (rd_kafka_t *rk, const char *topic, int32_t partition) |
RD_EXPORT rd_kafka_queue_t * | rd_kafka_queue_get_background (rd_kafka_t *rk) |
RD_EXPORT void | rd_kafka_queue_forward (rd_kafka_queue_t *src, rd_kafka_queue_t *dst) |
Forward/re-route queue src to dst . If dst is NULL the forwarding is removed. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_set_log_queue (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) |
Forward librdkafka logs (and debug) to the specified queue for serving with one of the ..poll() calls. More... | |
RD_EXPORT size_t | rd_kafka_queue_length (rd_kafka_queue_t *rkqu) |
RD_EXPORT void | rd_kafka_queue_io_event_enable (rd_kafka_queue_t *rkqu, int fd, const void *payload, size_t size) |
Enable IO event triggering for queue. More... | |
RD_EXPORT void | rd_kafka_queue_cb_event_enable (rd_kafka_queue_t *rkqu, void(*event_cb)(rd_kafka_t *rk, void *qev_opaque), void *qev_opaque) |
Enable callback event triggering for queue. More... | |
RD_EXPORT void | rd_kafka_queue_yield (rd_kafka_queue_t *rkqu) |
Cancels the current rd_kafka_queue_poll() on rkqu . More... | |
Simple Consumer API (legacy): Queue consumers | |
The following | |
RD_EXPORT rd_kafka_message_t * | rd_kafka_consume_queue (rd_kafka_queue_t *rkqu, int timeout_ms) |
Consume from queue. More... | |
RD_EXPORT ssize_t | rd_kafka_consume_batch_queue (rd_kafka_queue_t *rkqu, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size) |
Consume batch of messages from queue. More... | |
RD_EXPORT int | rd_kafka_consume_callback_queue (rd_kafka_queue_t *rkqu, int timeout_ms, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *commit_opaque), void *commit_opaque) |
Consume multiple messages from queue with callback. More... | |
Simple Consumer API (legacy): Topic+partition offset store. | |
If | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_offset_store (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset) |
Store offset offset + 1 for topic rkt partition partition . More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_offsets_store (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *offsets) |
Store offsets for next auto-commit for one or more partitions. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_offset_store_message (rd_kafka_message_t *rkmessage) |
Store offset +1 for the consumed message. More... | |
KafkaConsumer (C) | |
High-level KafkaConsumer C API | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_subscribe (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *topics) |
Subscribe to topic set using balanced consumer groups. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_unsubscribe (rd_kafka_t *rk) |
Unsubscribe from the current subscription set. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_subscription (rd_kafka_t *rk, rd_kafka_topic_partition_list_t **topics) |
Returns the current topic subscription. More... | |
RD_EXPORT rd_kafka_message_t * | rd_kafka_consumer_poll (rd_kafka_t *rk, int timeout_ms) |
Poll the consumer for messages or events. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_consumer_close (rd_kafka_t *rk) |
Close the consumer. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_consumer_close_queue (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) |
Asynchronously close the consumer. More... | |
RD_EXPORT int | rd_kafka_consumer_closed (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_error_t * | rd_kafka_incremental_assign (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions) |
Incrementally add partitions to the current assignment. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_incremental_unassign (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions) |
Incrementally remove partitions from the current assignment. More... | |
const RD_EXPORT char * | rd_kafka_rebalance_protocol (rd_kafka_t *rk) |
The rebalance protocol currently in use. This will be "NONE" if the consumer has not (yet) joined a group, else it will match the rebalance protocol ("EAGER", "COOPERATIVE") of the configured and selected assignor(s). All configured assignors must have the same protocol type, meaning online migration of a consumer group from using one protocol to another (in particular upgading from EAGER to COOPERATIVE) without a restart is not currently supported. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_assign (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *partitions) |
Atomic assignment of partitions to consume. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_assignment (rd_kafka_t *rk, rd_kafka_topic_partition_list_t **partitions) |
Returns the current partition assignment as set by rd_kafka_assign() or rd_kafka_incremental_assign(). More... | |
RD_EXPORT int | rd_kafka_assignment_lost (rd_kafka_t *rk) |
Check whether the consumer considers the current assignment to have been lost involuntarily. This method is only applicable for use with a high level subscribing consumer. Assignments are revoked immediately when determined to have been lost, so this method is only useful when reacting to a RD_KAFKA_EVENT_REBALANCE event or from within a rebalance_cb. Partitions that have been lost may already be owned by other members in the group and therefore commiting offsets, for example, may fail. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_commit (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, int async) |
Commit offsets on broker for the provided list of partitions. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, int async) |
Commit message's offset on broker for the message's partition. The committed offset is the message's offset + 1. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_commit_queue (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_queue_t *rkqu, void(*cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *commit_opaque), void *commit_opaque) |
Commit offsets on broker for the provided list of partitions. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_committed (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions, int timeout_ms) |
Retrieve committed offsets for topics+partitions. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_position (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions) |
Retrieve current positions (offsets) for topics+partitions. More... | |
RD_EXPORT rd_kafka_consumer_group_metadata_t * | rd_kafka_consumer_group_metadata (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_consumer_group_metadata_t * | rd_kafka_consumer_group_metadata_new (const char *group_id) |
Create a new consumer group metadata object. This is typically only used for writing tests. More... | |
RD_EXPORT rd_kafka_consumer_group_metadata_t * | rd_kafka_consumer_group_metadata_new_with_genid (const char *group_id, int32_t generation_id, const char *member_id, const char *group_instance_id) |
Create a new consumer group metadata object. This is typically only used for writing tests. More... | |
const RD_EXPORT char * | rd_kafka_consumer_group_metadata_member_id (const rd_kafka_consumer_group_metadata_t *group_metadata) |
Get member id of a group metadata. More... | |
RD_EXPORT void | rd_kafka_consumer_group_metadata_destroy (rd_kafka_consumer_group_metadata_t *) |
Frees the consumer group metadata object as returned by rd_kafka_consumer_group_metadata(). | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_consumer_group_metadata_write (const rd_kafka_consumer_group_metadata_t *cgmd, void **bufferp, size_t *sizep) |
Serialize the consumer group metadata to a binary format. This is mainly for client binding use and not for application use. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_consumer_group_metadata_read (rd_kafka_consumer_group_metadata_t **cgmdp, const void *buffer, size_t size) |
Reads serialized consumer group metadata and returns a consumer group metadata object. This is mainly for client binding use and not for application use. More... | |
Experimental APIs | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_poll_set_consumer (rd_kafka_t *rk) |
Redirect the main (rd_kafka_poll()) queue to the KafkaConsumer's queue (rd_kafka_consumer_poll()). More... | |
Auxiliary types | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_topic_result_error (const rd_kafka_topic_result_t *topicres) |
Topic result provides per-topic operation result information. More... | |
const RD_EXPORT char * | rd_kafka_topic_result_error_string (const rd_kafka_topic_result_t *topicres) |
const RD_EXPORT char * | rd_kafka_topic_result_name (const rd_kafka_topic_result_t *topicres) |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_group_result_error (const rd_kafka_group_result_t *groupres) |
Group result provides per-group operation result information. More... | |
const RD_EXPORT char * | rd_kafka_group_result_name (const rd_kafka_group_result_t *groupres) |
const RD_EXPORT rd_kafka_topic_partition_list_t * | rd_kafka_group_result_partitions (const rd_kafka_group_result_t *groupres) |
const RD_EXPORT rd_kafka_topic_partition_t * | rd_kafka_topic_partition_result_partition (const rd_kafka_topic_partition_result_t *partition_result) |
Topic Partition Result provides per-topic+partition operation result Consists of TopicPartition object and error object. More... | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_topic_partition_result_error (const rd_kafka_topic_partition_result_t *partition_result) |
Admin API - DescribeCluster | |
RD_EXPORT void | rd_kafka_DescribeCluster (rd_kafka_t *rk, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Describes the cluster. More... | |
const RD_EXPORT rd_kafka_Node_t ** | rd_kafka_DescribeCluster_result_nodes (const rd_kafka_DescribeCluster_result_t *result, size_t *cntp) |
Gets the broker nodes for the result cluster. More... | |
const RD_EXPORT rd_kafka_AclOperation_t * | rd_kafka_DescribeCluster_result_authorized_operations (const rd_kafka_DescribeCluster_result_t *result, size_t *cntp) |
Gets the authorized ACL operations for the result cluster. More... | |
const RD_EXPORT rd_kafka_Node_t * | rd_kafka_DescribeCluster_result_controller (const rd_kafka_DescribeCluster_result_t *result) |
Gets the current controller for the result cluster. More... | |
const RD_EXPORT char * | rd_kafka_DescribeCluster_result_cluster_id (const rd_kafka_DescribeCluster_result_t *result) |
Gets the cluster id for the result cluster. More... | |
Security APIs | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_oauthbearer_set_token (rd_kafka_t *rk, const char *token_value, int64_t md_lifetime_ms, const char *md_principal_name, const char **extensions, size_t extension_size, char *errstr, size_t errstr_size) |
Set SASL/OAUTHBEARER token and metadata. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_oauthbearer_set_token_failure (rd_kafka_t *rk, const char *errstr) |
SASL/OAUTHBEARER token refresh failure indicator. More... | |
Transactional producer API | |
The transactional producer operates on top of the idempotent producer, and provides full exactly-once semantics (EOS) for Apache Kafka when used with the transaction aware consumer ( A producer instance is configured for transactions by setting the After creating the transactional producer instance using rd_kafka_new() the transactional state must be initialized by calling rd_kafka_init_transactions(). This is a blocking call that will acquire a runtime producer id from the transaction coordinator broker as well as abort any stale transactions and fence any still running producer instances with the same Once transactions are initialized the application may begin a new transaction by calling rd_kafka_begin_transaction(). A producer instance may only have one single on-going transaction. Any messages produced after the transaction has been started will belong to the ongoing transaction and will be committed or aborted atomically. It is not permitted to produce messages outside a transaction boundary, e.g., before rd_kafka_begin_transaction() or after rd_kafka_commit_transaction(), rd_kafka_abort_transaction(), or after the current transaction has failed. If consumed messages are used as input to the transaction, the consumer instance must be configured with To commit the produced messages, and any consumed offsets, to the current transaction, call rd_kafka_commit_transaction(). This call will block until the transaction has been fully committed or failed (typically due to fencing by a newer producer instance). Alternatively, if processing fails, or an abortable transaction error is raised, the transaction needs to be aborted by calling rd_kafka_abort_transaction() which marks any produced messages and offset commits as aborted. After the current transaction has been committed or aborted a new transaction may be started by calling rd_kafka_begin_transaction() again.
| |
RD_EXPORT rd_kafka_error_t * | rd_kafka_init_transactions (rd_kafka_t *rk, int timeout_ms) |
Initialize transactions for the producer instance. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_begin_transaction (rd_kafka_t *rk) |
Begin a new transaction. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_send_offsets_to_transaction (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, const rd_kafka_consumer_group_metadata_t *cgmetadata, int timeout_ms) |
Sends a list of topic partition offsets to the consumer group coordinator for cgmetadata , and marks the offsets as part part of the current transaction. These offsets will be considered committed only if the transaction is committed successfully. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_commit_transaction (rd_kafka_t *rk, int timeout_ms) |
Commit the current transaction (as started with rd_kafka_begin_transaction()). More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_abort_transaction (rd_kafka_t *rk, int timeout_ms) |
Aborts the ongoing transaction. More... | |
librdkafka version | |
#define | RD_KAFKA_VERSION 0x020600ff |
librdkafka version More... | |
RD_EXPORT int | rd_kafka_version (void) |
Returns the librdkafka version as integer. More... | |
const RD_EXPORT char * | rd_kafka_version_str (void) |
Returns the librdkafka version as string. More... | |
Var-arg tag types | |
#define | RD_KAFKA_V_END RD_KAFKA_VTYPE_END |
Convenience macros for rd_kafka_vtype_t that takes the correct arguments for each vtype. More... | |
#define | RD_KAFKA_V_TOPIC(topic) |
#define | RD_KAFKA_V_RKT(rkt) |
#define | RD_KAFKA_V_PARTITION(partition) |
#define | RD_KAFKA_V_VALUE(VALUE, LEN) |
#define | RD_KAFKA_V_KEY(KEY, LEN) |
#define | RD_KAFKA_V_OPAQUE(msg_opaque) |
#define | RD_KAFKA_V_MSGFLAGS(msgflags) _LRK_TYPECHECK(RD_KAFKA_VTYPE_MSGFLAGS, int, msgflags), (int)msgflags |
#define | RD_KAFKA_V_TIMESTAMP(timestamp) |
#define | RD_KAFKA_V_HEADER(NAME, VALUE, LEN) |
#define | RD_KAFKA_V_HEADERS(HDRS) |
enum | rd_kafka_vtype_t { RD_KAFKA_VTYPE_END, RD_KAFKA_VTYPE_TOPIC, RD_KAFKA_VTYPE_RKT, RD_KAFKA_VTYPE_PARTITION, RD_KAFKA_VTYPE_VALUE, RD_KAFKA_VTYPE_KEY, RD_KAFKA_VTYPE_OPAQUE, RD_KAFKA_VTYPE_MSGFLAGS, RD_KAFKA_VTYPE_TIMESTAMP, RD_KAFKA_VTYPE_HEADER, RD_KAFKA_VTYPE_HEADERS } |
Var-arg tag types. More... | |
Kafka messages | |
enum | rd_kafka_msg_status_t { RD_KAFKA_MSG_STATUS_NOT_PERSISTED = 0, RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED = 1, RD_KAFKA_MSG_STATUS_PERSISTED = 2 } |
Message persistence status can be used by the application to find out if a produced message was persisted in the topic log. More... | |
RD_EXPORT void | rd_kafka_message_destroy (rd_kafka_message_t *rkmessage) |
Frees resources for rkmessage and hands ownership back to rdkafka. | |
const RD_EXPORT char * | rd_kafka_message_errstr (const rd_kafka_message_t *rkmessage) |
Returns the error string for an errored rd_kafka_message_t or NULL if there was no error. More... | |
const RD_EXPORT char * | rd_kafka_message_produce_errstr (const rd_kafka_message_t *rkmessage) |
Returns the error string for an errored produced rd_kafka_message_t or NULL if there was no error. More... | |
RD_EXPORT int64_t | rd_kafka_message_timestamp (const rd_kafka_message_t *rkmessage, rd_kafka_timestamp_type_t *tstype) |
Returns the message timestamp for a consumed message. More... | |
RD_EXPORT int64_t | rd_kafka_message_latency (const rd_kafka_message_t *rkmessage) |
Returns the latency for a produced message measured from the produce() call. More... | |
RD_EXPORT int32_t | rd_kafka_message_broker_id (const rd_kafka_message_t *rkmessage) |
Returns the broker id of the broker the message was produced to or fetched from. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_message_headers (const rd_kafka_message_t *rkmessage, rd_kafka_headers_t **hdrsp) |
Get the message header list. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_message_detach_headers (rd_kafka_message_t *rkmessage, rd_kafka_headers_t **hdrsp) |
Get the message header list and detach the list from the message making the application the owner of the headers. The application must eventually destroy the headers using rd_kafka_headers_destroy(). The message's headers will be set to NULL. More... | |
RD_EXPORT void | rd_kafka_message_set_headers (rd_kafka_message_t *rkmessage, rd_kafka_headers_t *hdrs) |
Replace the message's current headers with a new list. More... | |
RD_EXPORT size_t | rd_kafka_header_cnt (const rd_kafka_headers_t *hdrs) |
Returns the number of header key/value pairs. More... | |
RD_EXPORT rd_kafka_msg_status_t | rd_kafka_message_status (const rd_kafka_message_t *rkmessage) |
Returns the message's persistence status in the topic log. More... | |
RD_EXPORT int32_t | rd_kafka_message_leader_epoch (const rd_kafka_message_t *rkmessage) |
Configuration interface | |
enum | rd_kafka_conf_res_t { RD_KAFKA_CONF_UNKNOWN = -2, RD_KAFKA_CONF_INVALID = -1, RD_KAFKA_CONF_OK = 0 } |
Configuration result type. More... | |
enum | rd_kafka_cert_type_t { RD_KAFKA_CERT_PUBLIC_KEY, RD_KAFKA_CERT_PRIVATE_KEY, RD_KAFKA_CERT_CA, RD_KAFKA_CERT__CNT } |
SSL certificate type. More... | |
enum | rd_kafka_cert_enc_t { RD_KAFKA_CERT_ENC_PKCS12, RD_KAFKA_CERT_ENC_DER, RD_KAFKA_CERT_ENC_PEM, RD_KAFKA_CERT_ENC__CNT } |
SSL certificate encoding. More... | |
RD_EXPORT rd_kafka_conf_t * | rd_kafka_conf_new (void) |
Create configuration object. More... | |
RD_EXPORT void | rd_kafka_conf_destroy (rd_kafka_conf_t *conf) |
Destroys a conf object. | |
RD_EXPORT rd_kafka_conf_t * | rd_kafka_conf_dup (const rd_kafka_conf_t *conf) |
Creates a copy/duplicate of configuration object conf . More... | |
RD_EXPORT rd_kafka_conf_t * | rd_kafka_conf_dup_filter (const rd_kafka_conf_t *conf, size_t filter_cnt, const char **filter) |
Same as rd_kafka_conf_dup() but with an array of property name prefixes to filter out (ignore) when copying. | |
const RD_EXPORT rd_kafka_conf_t * | rd_kafka_conf (rd_kafka_t *rk) |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_conf_set (rd_kafka_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size) |
Sets a configuration property. More... | |
RD_EXPORT void | rd_kafka_conf_set_events (rd_kafka_conf_t *conf, int events) |
Enable event sourcing. events is a bitmask of RD_KAFKA_EVENT_* of events to enable for consumption by rd_kafka_queue_poll() . | |
RD_EXPORT void | rd_kafka_conf_set_background_event_cb (rd_kafka_conf_t *conf, void(*event_cb)(rd_kafka_t *rk, rd_kafka_event_t *rkev, void *opaque)) |
Generic event callback to be used with the event API to trigger callbacks for rd_kafka_event_t objects from a background thread serving the background queue. More... | |
RD_EXPORT void | rd_kafka_conf_set_dr_cb (rd_kafka_conf_t *conf, void(*dr_cb)(rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque)) |
RD_EXPORT void | rd_kafka_conf_set_dr_msg_cb (rd_kafka_conf_t *conf, void(*dr_msg_cb)(rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, void *opaque)) |
Producer: Set delivery report callback in provided conf object. More... | |
RD_EXPORT void | rd_kafka_conf_set_consume_cb (rd_kafka_conf_t *conf, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *opaque)) |
Consumer: Set consume callback for use with rd_kafka_consumer_poll() More... | |
RD_EXPORT void | rd_kafka_conf_set_rebalance_cb (rd_kafka_conf_t *conf, void(*rebalance_cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *partitions, void *opaque)) |
Consumer: Set rebalance callback for use with coordinated consumer group balancing. More... | |
RD_EXPORT void | rd_kafka_conf_set_offset_commit_cb (rd_kafka_conf_t *conf, void(*offset_commit_cb)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *opaque)) |
Consumer: Set offset commit callback for use with consumer groups. More... | |
RD_EXPORT void | rd_kafka_conf_set_error_cb (rd_kafka_conf_t *conf, void(*error_cb)(rd_kafka_t *rk, int err, const char *reason, void *opaque)) |
Set error callback in provided conf object. More... | |
RD_EXPORT void | rd_kafka_conf_set_throttle_cb (rd_kafka_conf_t *conf, void(*throttle_cb)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int throttle_time_ms, void *opaque)) |
Set throttle callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_log_cb (rd_kafka_conf_t *conf, void(*log_cb)(const rd_kafka_t *rk, int level, const char *fac, const char *buf)) |
Set logger callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_stats_cb (rd_kafka_conf_t *conf, int(*stats_cb)(rd_kafka_t *rk, char *json, size_t json_len, void *opaque)) |
Set statistics callback in provided conf object. More... | |
RD_EXPORT void | rd_kafka_conf_set_oauthbearer_token_refresh_cb (rd_kafka_conf_t *conf, void(*oauthbearer_token_refresh_cb)(rd_kafka_t *rk, const char *oauthbearer_config, void *opaque)) |
Set SASL/OAUTHBEARER token refresh callback in provided conf object. More... | |
RD_EXPORT void | rd_kafka_conf_enable_sasl_queue (rd_kafka_conf_t *conf, int enable) |
Enable/disable creation of a queue specific to SASL events and callbacks. More... | |
RD_EXPORT void | rd_kafka_conf_set_socket_cb (rd_kafka_conf_t *conf, int(*socket_cb)(int domain, int type, int protocol, void *opaque)) |
Set socket callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_connect_cb (rd_kafka_conf_t *conf, int(*connect_cb)(int sockfd, const struct sockaddr *addr, int addrlen, const char *id, void *opaque)) |
Set connect callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_closesocket_cb (rd_kafka_conf_t *conf, int(*closesocket_cb)(int sockfd, void *opaque)) |
Set close socket callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_open_cb (rd_kafka_conf_t *conf, int(*open_cb)(const char *pathname, int flags, mode_t mode, void *opaque)) |
Set open callback. More... | |
RD_EXPORT void | rd_kafka_conf_set_resolve_cb (rd_kafka_conf_t *conf, int(*resolve_cb)(const char *node, const char *service, const struct addrinfo *hints, struct addrinfo **res, void *opaque)) |
Set address resolution callback. More... | |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_conf_set_ssl_cert_verify_cb (rd_kafka_conf_t *conf, int(*ssl_cert_verify_cb)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int *x509_error, int depth, const char *buf, size_t size, char *errstr, size_t errstr_size, void *opaque)) |
Sets the verification callback of the broker certificate. More... | |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_conf_set_ssl_cert (rd_kafka_conf_t *conf, rd_kafka_cert_type_t cert_type, rd_kafka_cert_enc_t cert_enc, const void *buffer, size_t size, char *errstr, size_t errstr_size) |
Set certificate/key cert_type from the cert_enc encoded memory at buffer of size bytes. More... | |
RD_EXPORT void | rd_kafka_conf_set_engine_callback_data (rd_kafka_conf_t *conf, void *callback_data) |
Set callback_data for OpenSSL engine. More... | |
RD_EXPORT void | rd_kafka_conf_set_opaque (rd_kafka_conf_t *conf, void *opaque) |
Sets the application's opaque pointer that will be passed to callbacks. More... | |
RD_EXPORT void * | rd_kafka_opaque (const rd_kafka_t *rk) |
Retrieves the opaque pointer previously set with rd_kafka_conf_set_opaque() | |
RD_EXPORT void | rd_kafka_conf_set_default_topic_conf (rd_kafka_conf_t *conf, rd_kafka_topic_conf_t *tconf) |
Sets the default topic configuration to use for automatically subscribed topics (e.g., through pattern-matched topics). The topic config object is not usable after this call. More... | |
RD_EXPORT rd_kafka_topic_conf_t * | rd_kafka_conf_get_default_topic_conf (rd_kafka_conf_t *conf) |
Gets the default topic configuration as previously set with rd_kafka_conf_set_default_topic_conf() or that was implicitly created by configuring a topic-level property on the global conf object. More... | |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_conf_get (const rd_kafka_conf_t *conf, const char *name, char *dest, size_t *dest_size) |
Retrieve configuration value for property name . More... | |
RD_EXPORT rd_kafka_conf_res_t | rd_kafka_topic_conf_get (const rd_kafka_topic_conf_t *conf, const char *name, char *dest, size_t *dest_size) |
Retrieve topic configuration value for property name . More... | |
const RD_EXPORT char ** | rd_kafka_conf_dump (rd_kafka_conf_t *conf, size_t *cntp) |
Dump the configuration properties and values of conf to an array with "key", "value" pairs. More... | |
const RD_EXPORT char ** | rd_kafka_topic_conf_dump (rd_kafka_topic_conf_t *conf, size_t *cntp) |
Dump the topic configuration properties and values of conf to an array with "key", "value" pairs. More... | |
RD_EXPORT void | rd_kafka_conf_dump_free (const char **arr, size_t cnt) |
Frees a configuration dump returned from rd_kafka_conf_dump() or `rd_kafka_topic_conf_dump(). | |
RD_EXPORT void | rd_kafka_conf_properties_show (FILE *fp) |
Prints a table to fp of all supported configuration properties, their default values as well as a description. More... | |
Main Kafka and Topic object handles | |
#define | RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE 0x8 |
Flags for rd_kafka_destroy_flags() More... | |
#define | RD_KAFKA_PARTITION_UA ((int32_t)-1) |
Unassigned partition. More... | |
RD_EXPORT rd_kafka_t * | rd_kafka_new (rd_kafka_type_t type, rd_kafka_conf_t *conf, char *errstr, size_t errstr_size) |
Creates a new Kafka handle and starts its operation according to the specified type (RD_KAFKA_CONSUMER or RD_KAFKA_PRODUCER ). More... | |
RD_EXPORT void | rd_kafka_destroy (rd_kafka_t *rk) |
Destroy Kafka handle. More... | |
RD_EXPORT void | rd_kafka_destroy_flags (rd_kafka_t *rk, int flags) |
Destroy Kafka handle according to specified destroy flags. More... | |
const RD_EXPORT char * | rd_kafka_name (const rd_kafka_t *rk) |
Returns Kafka handle name. | |
RD_EXPORT rd_kafka_type_t | rd_kafka_type (const rd_kafka_t *rk) |
Returns Kafka handle type. | |
RD_EXPORT char * | rd_kafka_memberid (const rd_kafka_t *rk) |
Returns this client's broker-assigned group member id. More... | |
RD_EXPORT char * | rd_kafka_clusterid (rd_kafka_t *rk, int timeout_ms) |
Returns the ClusterId as reported in broker metadata. More... | |
RD_EXPORT int32_t | rd_kafka_controllerid (rd_kafka_t *rk, int timeout_ms) |
Returns the current ControllerId as reported in broker metadata. More... | |
RD_EXPORT rd_kafka_topic_t * | rd_kafka_topic_new (rd_kafka_t *rk, const char *topic, rd_kafka_topic_conf_t *conf) |
Creates a new topic handle for topic named topic . More... | |
RD_EXPORT void | rd_kafka_topic_destroy (rd_kafka_topic_t *rkt) |
Loose application's topic handle refcount as previously created with rd_kafka_topic_new() . More... | |
const RD_EXPORT char * | rd_kafka_topic_name (const rd_kafka_topic_t *rkt) |
Returns the topic name. | |
RD_EXPORT void * | rd_kafka_topic_opaque (const rd_kafka_topic_t *rkt) |
Get the rkt_opaque pointer that was set in the topic configuration with rd_kafka_topic_conf_set_opaque(). | |
RD_EXPORT int | rd_kafka_poll (rd_kafka_t *rk, int timeout_ms) |
Polls the provided kafka handle for events. More... | |
RD_EXPORT void | rd_kafka_yield (rd_kafka_t *rk) |
Cancels the current callback dispatcher (rd_kafka_poll(), rd_kafka_consume_callback(), etc). More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_pause_partitions (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions) |
Pause producing or consumption for the provided list of partitions. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_resume_partitions (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions) |
Resume producing consumption for the provided list of partitions. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_query_watermark_offsets (rd_kafka_t *rk, const char *topic, int32_t partition, int64_t *low, int64_t *high, int timeout_ms) |
Query broker for low (oldest/beginning) and high (newest/end) offsets for partition. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_get_watermark_offsets (rd_kafka_t *rk, const char *topic, int32_t partition, int64_t *low, int64_t *high) |
Get last known low (oldest/beginning) and high (newest/end) offsets for partition. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_offsets_for_times (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *offsets, int timeout_ms) |
Look up the offsets for the given partitions by timestamp. More... | |
RD_EXPORT void * | rd_kafka_mem_calloc (rd_kafka_t *rk, size_t num, size_t size) |
Allocate and zero memory using the same allocator librdkafka uses. More... | |
RD_EXPORT void * | rd_kafka_mem_malloc (rd_kafka_t *rk, size_t size) |
Allocate memory using the same allocator librdkafka uses. More... | |
RD_EXPORT void | rd_kafka_mem_free (rd_kafka_t *rk, void *ptr) |
Free pointer returned by librdkafka. More... | |
Simple Consumer API (legacy) | |
#define | RD_KAFKA_OFFSET_BEGINNING -2 |
#define | RD_KAFKA_OFFSET_END -1 |
#define | RD_KAFKA_OFFSET_STORED -1000 |
#define | RD_KAFKA_OFFSET_INVALID -1001 |
#define | RD_KAFKA_OFFSET_TAIL(CNT) (RD_KAFKA_OFFSET_TAIL_BASE - (CNT)) |
Start consuming CNT messages from topic's current end offset. More... | |
RD_EXPORT int | rd_kafka_consume_start (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset) |
Start consuming messages for topic rkt and partition at offset offset which may either be an absolute (0..N) or one of the logical offsets: More... | |
RD_EXPORT int | rd_kafka_consume_start_queue (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset, rd_kafka_queue_t *rkqu) |
Same as rd_kafka_consume_start() but re-routes incoming messages to the provided queue rkqu (which must have been previously allocated with rd_kafka_queue_new() . More... | |
RD_EXPORT int | rd_kafka_consume_stop (rd_kafka_topic_t *rkt, int32_t partition) |
Stop consuming messages for topic rkt and partition , purging all messages currently in the local queue. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_seek (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset, int timeout_ms) |
Seek consumer for topic+partition to offset which is either an absolute or logical offset. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_seek_partitions (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions, int timeout_ms) |
Seek consumer for partitions in partitions to the per-partition offset in the .offset field of partitions . More... | |
RD_EXPORT rd_kafka_message_t * | rd_kafka_consume (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms) |
Consume a single message from topic rkt and partition . More... | |
RD_EXPORT ssize_t | rd_kafka_consume_batch (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size) |
Consume up to rkmessages_size from topic rkt and partition putting a pointer to each message in the application provided array rkmessages (of size rkmessages_size entries). More... | |
RD_EXPORT int | rd_kafka_consume_callback (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, void(*consume_cb)(rd_kafka_message_t *rkmessage, void *commit_opaque), void *commit_opaque) |
Consumes messages from topic rkt and partition , calling the provided callback for each consumed messsage. More... | |
Producer API | |
#define | RD_KAFKA_MSG_F_FREE 0x1 |
Producer message flags. More... | |
#define | RD_KAFKA_MSG_F_COPY 0x2 |
#define | RD_KAFKA_MSG_F_BLOCK 0x4 |
#define | RD_KAFKA_MSG_F_PARTITION 0x8 |
#define | RD_KAFKA_PURGE_F_QUEUE 0x1 |
Flags for rd_kafka_purge() More... | |
#define | RD_KAFKA_PURGE_F_INFLIGHT 0x2 |
#define | RD_KAFKA_PURGE_F_NON_BLOCKING 0x4 |
RD_EXPORT int | rd_kafka_produce (rd_kafka_topic_t *rkt, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t keylen, void *msg_opaque) |
Produce and send a single message to broker. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_producev (rd_kafka_t *rk,...) |
Produce and send a single message to broker. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_produceva (rd_kafka_t *rk, const rd_kafka_vu_t *vus, size_t cnt) |
Produce and send a single message to broker. More... | |
RD_EXPORT int | rd_kafka_produce_batch (rd_kafka_topic_t *rkt, int32_t partition, int msgflags, rd_kafka_message_t *rkmessages, int message_cnt) |
Produce multiple messages. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_flush (rd_kafka_t *rk, int timeout_ms) |
Wait until all outstanding produce requests, et.al, are completed. This should typically be done prior to destroying a producer instance to make sure all queued and in-flight produce requests are completed before terminating. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_purge (rd_kafka_t *rk, int purge_flags) |
Purge messages currently handled by the producer instance. More... | |
Metadata API | |
typedef struct rd_kafka_Node_s | rd_kafka_Node_t |
Node (broker) information. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_metadata (rd_kafka_t *rk, int all_topics, rd_kafka_topic_t *only_rkt, const struct rd_kafka_metadata **metadatap, int timeout_ms) |
Request Metadata from broker. More... | |
RD_EXPORT void | rd_kafka_metadata_destroy (const struct rd_kafka_metadata *metadata) |
Release metadata memory. | |
RD_EXPORT int | rd_kafka_Node_id (const rd_kafka_Node_t *node) |
Get the id of node . More... | |
const RD_EXPORT char * | rd_kafka_Node_host (const rd_kafka_Node_t *node) |
Get the host of node . More... | |
RD_EXPORT uint16_t | rd_kafka_Node_port (const rd_kafka_Node_t *node) |
Get the port of node . More... | |
const RD_EXPORT char * | rd_kafka_Node_rack (const rd_kafka_Node_t *node) |
Get the rack of node . More... | |
Client group information | |
enum | rd_kafka_consumer_group_state_t { RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN = 0, RD_KAFKA_CONSUMER_GROUP_STATE_PREPARING_REBALANCE = 1, RD_KAFKA_CONSUMER_GROUP_STATE_COMPLETING_REBALANCE = 2, RD_KAFKA_CONSUMER_GROUP_STATE_STABLE = 3, RD_KAFKA_CONSUMER_GROUP_STATE_DEAD = 4, RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY = 5, RD_KAFKA_CONSUMER_GROUP_STATE__CNT } |
Consumer group state. | |
enum | rd_kafka_consumer_group_type_t { RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN = 0, RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER = 1, RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC = 2, RD_KAFKA_CONSUMER_GROUP_TYPE__CNT } |
Consumer group type. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_list_groups (rd_kafka_t *rk, const char *group, const struct rd_kafka_group_list **grplistp, int timeout_ms) |
List and describe client groups in cluster. More... | |
const RD_EXPORT char * | rd_kafka_consumer_group_state_name (rd_kafka_consumer_group_state_t state) |
Returns a name for a state code. More... | |
RD_EXPORT rd_kafka_consumer_group_state_t | rd_kafka_consumer_group_state_code (const char *name) |
Returns a code for a state name. More... | |
const RD_EXPORT char * | rd_kafka_consumer_group_type_name (rd_kafka_consumer_group_type_t type) |
Returns a name for a group type code. More... | |
RD_EXPORT rd_kafka_consumer_group_type_t | rd_kafka_consumer_group_type_code (const char *name) |
Returns a code for a group type name. More... | |
RD_EXPORT void | rd_kafka_group_list_destroy (const struct rd_kafka_group_list *grplist) |
Release list memory. | |
Miscellaneous APIs | |
enum | rd_kafka_thread_type_t { RD_KAFKA_THREAD_MAIN, RD_KAFKA_THREAD_BACKGROUND, RD_KAFKA_THREAD_BROKER } |
librdkafka internal thread type. More... | |
RD_EXPORT int | rd_kafka_brokers_add (rd_kafka_t *rk, const char *brokerlist) |
Adds one or more brokers to the kafka handle's list of initial bootstrap brokers. More... | |
RD_EXPORT RD_DEPRECATED void | rd_kafka_set_logger (rd_kafka_t *rk, void(*func)(const rd_kafka_t *rk, int level, const char *fac, const char *buf)) |
Set logger function. More... | |
RD_EXPORT void | rd_kafka_set_log_level (rd_kafka_t *rk, int level) |
Specifies the maximum logging level emitted by internal kafka logging and debugging. More... | |
RD_EXPORT void | rd_kafka_log_print (const rd_kafka_t *rk, int level, const char *fac, const char *buf) |
Builtin (default) log sink: print to stderr. | |
RD_EXPORT void | rd_kafka_log_syslog (const rd_kafka_t *rk, int level, const char *fac, const char *buf) |
Builtin log sink: print to syslog. More... | |
RD_EXPORT int | rd_kafka_outq_len (rd_kafka_t *rk) |
Returns the current out queue length. More... | |
RD_EXPORT void | rd_kafka_dump (FILE *fp, rd_kafka_t *rk) |
Dumps rdkafka's internal state for handle rk to stream fp . More... | |
RD_EXPORT int | rd_kafka_thread_cnt (void) |
Retrieve the current number of threads in use by librdkafka. More... | |
RD_EXPORT int | rd_kafka_wait_destroyed (int timeout_ms) |
Wait for all rd_kafka_t objects to be destroyed. More... | |
RD_EXPORT int | rd_kafka_unittest (void) |
Run librdkafka's built-in unit-tests. More... | |
Interceptors | |
A callback interface that allows message interception for both producer and consumer data pipelines. Except for the on_new(), on_conf_set(), on_conf_dup() and on_conf_destroy() interceptors, interceptors are added to the newly created rd_kafka_t client instance. These interceptors MUST only be added from on_new() and MUST NOT be added after rd_kafka_new() returns. The on_new(), on_conf_set(), on_conf_dup() and on_conf_destroy() interceptors are added to the configuration object which is later passed to rd_kafka_new() where on_new() is called to allow addition of other interceptors. Each interceptor reference consists of a display name (ic_name), a callback function, and an application-specified opaque value that is passed as-is to the callback. The ic_name must be unique for the interceptor implementation and is used to reject duplicate interceptor methods. Any number of interceptors can be added and they are called in the order they were added, unless otherwise noted. The list of registered interceptor methods are referred to as interceptor chains.
| |
typedef rd_kafka_conf_res_t() | rd_kafka_interceptor_f_on_conf_set_t(rd_kafka_conf_t *conf, const char *name, const char *val, char *errstr, size_t errstr_size, void *ic_opaque) |
on_conf_set() is called from rd_kafka_*_conf_set() in the order the interceptors were added. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_conf_dup_t(rd_kafka_conf_t *new_conf, const rd_kafka_conf_t *old_conf, size_t filter_cnt, const char **filter, void *ic_opaque) |
on_conf_dup() is called from rd_kafka_conf_dup() in the order the interceptors were added and is used to let an interceptor re-register its conf interecptors with a new opaque value. The on_conf_dup() method is called prior to the configuration from old_conf being copied to new_conf . More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_conf_destroy_t(void *ic_opaque) |
on_conf_destroy() is called from rd_kafka_*_conf_destroy() in the order the interceptors were added. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_new_t(rd_kafka_t *rk, const rd_kafka_conf_t *conf, void *ic_opaque, char *errstr, size_t errstr_size) |
on_new() is called from rd_kafka_new() prior toreturning the newly created client instance to the application. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_destroy_t(rd_kafka_t *rk, void *ic_opaque) |
on_destroy() is called from rd_kafka_destroy() or (rd_kafka_new() if rd_kafka_new() fails during initialization). More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_send_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_send() is called from rd_kafka_produce*() (et.al) prior to the partitioner being called. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_acknowledgement_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_acknowledgement() is called to inform interceptors that a message was succesfully delivered or permanently failed delivery. The interceptor chain is called from internal librdkafka background threads, or rd_kafka_produce*() if the partitioner failed. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_consume_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_consume() is called just prior to passing the message to the application in rd_kafka_consumer_poll(), rd_kafka_consume*(), the event interface, etc. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_commit_t(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_resp_err_t err, void *ic_opaque) |
on_commit() is called on completed or failed offset commit. It is called from internal librdkafka threads. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_request_sent_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, void *ic_opaque) |
on_request_sent() is called when a request has been fully written to a broker TCP connections socket. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_response_received_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, int64_t rtt, rd_kafka_resp_err_t err, void *ic_opaque) |
on_response_received() is called when a protocol response has been fully received from a broker TCP connection socket but before the response payload is parsed. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_thread_start_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque) |
on_thread_start() is called from a newly created librdkafka-managed thread. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_thread_exit_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque) |
on_thread_exit() is called just prior to a librdkafka-managed thread exiting from the exiting thread itself. More... | |
typedef rd_kafka_resp_err_t() | rd_kafka_interceptor_f_on_broker_state_change_t(rd_kafka_t *rk, int32_t broker_id, const char *secproto, const char *name, int port, const char *state, void *ic_opaque) |
on_broker_state_change() is called just after a broker has been created or its state has been changed. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_conf_interceptor_add_on_conf_set (rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_set_t *on_conf_set, void *ic_opaque) |
Append an on_conf_set() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_conf_interceptor_add_on_conf_dup (rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_dup_t *on_conf_dup, void *ic_opaque) |
Append an on_conf_dup() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_conf_interceptor_add_on_conf_destroy (rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_conf_destroy_t *on_conf_destroy, void *ic_opaque) |
Append an on_conf_destroy() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_conf_interceptor_add_on_new (rd_kafka_conf_t *conf, const char *ic_name, rd_kafka_interceptor_f_on_new_t *on_new, void *ic_opaque) |
Append an on_new() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_destroy (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_destroy_t *on_destroy, void *ic_opaque) |
Append an on_destroy() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_send (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_send_t *on_send, void *ic_opaque) |
Append an on_send() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_acknowledgement (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_acknowledgement_t *on_acknowledgement, void *ic_opaque) |
Append an on_acknowledgement() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_consume (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_consume_t *on_consume, void *ic_opaque) |
Append an on_consume() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_commit (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_commit_t *on_commit, void *ic_opaque) |
Append an on_commit() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_request_sent (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_request_sent_t *on_request_sent, void *ic_opaque) |
Append an on_request_sent() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_response_received (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_response_received_t *on_response_received, void *ic_opaque) |
Append an on_response_received() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_thread_start (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_thread_start_t *on_thread_start, void *ic_opaque) |
Append an on_thread_start() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_thread_exit (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_thread_exit_t *on_thread_exit, void *ic_opaque) |
Append an on_thread_exit() interceptor. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_interceptor_add_on_broker_state_change (rd_kafka_t *rk, const char *ic_name, rd_kafka_interceptor_f_on_broker_state_change_t *on_broker_state_change, void *ic_opaque) |
Append an on_broker_state_change() interceptor. More... | |
Admin API | |
The Admin API enables applications to perform administrative Apache Kafka tasks, such as creating and deleting topics, altering and reading broker configuration, etc. The Admin API is asynchronous and makes use of librdkafka's standard Use Use rd_kafka_event_error() and rd_kafka_event_error_string() to acquire the request-level error/success for an Admin API request. Even if the returned value is Locally triggered errors:
| |
enum | rd_kafka_admin_op_t { RD_KAFKA_ADMIN_OP_ANY = 0, RD_KAFKA_ADMIN_OP_CREATETOPICS, RD_KAFKA_ADMIN_OP_DELETETOPICS, RD_KAFKA_ADMIN_OP_CREATEPARTITIONS, RD_KAFKA_ADMIN_OP_ALTERCONFIGS, RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS, RD_KAFKA_ADMIN_OP_DELETERECORDS, RD_KAFKA_ADMIN_OP_DELETEGROUPS, RD_KAFKA_ADMIN_OP_DELETECONSUMERGROUPOFFSETS, RD_KAFKA_ADMIN_OP_CREATEACLS, RD_KAFKA_ADMIN_OP_DESCRIBEACLS, RD_KAFKA_ADMIN_OP_DELETEACLS, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS, RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPOFFSETS, RD_KAFKA_ADMIN_OP_ALTERCONSUMERGROUPOFFSETS, RD_KAFKA_ADMIN_OP_INCREMENTALALTERCONFIGS, RD_KAFKA_ADMIN_OP_DESCRIBEUSERSCRAMCREDENTIALS, RD_KAFKA_ADMIN_OP_ALTERUSERSCRAMCREDENTIALS, RD_KAFKA_ADMIN_OP_DESCRIBETOPICS, RD_KAFKA_ADMIN_OP_DESCRIBECLUSTER, RD_KAFKA_ADMIN_OP_LISTOFFSETS, RD_KAFKA_ADMIN_OP_ELECTLEADERS, RD_KAFKA_ADMIN_OP__CNT } |
Admin operation enum name for use with rd_kafka_AdminOptions_new() More... | |
enum | rd_kafka_IsolationLevel_t { RD_KAFKA_ISOLATION_LEVEL_READ_UNCOMMITTED = 0, RD_KAFKA_ISOLATION_LEVEL_READ_COMMITTED = 1 } |
IsolationLevel enum name for use with rd_kafka_AdminOptions_new() More... | |
enum | rd_kafka_AclOperation_t { RD_KAFKA_ACL_OPERATION_UNKNOWN = 0, RD_KAFKA_ACL_OPERATION_ANY, RD_KAFKA_ACL_OPERATION_ALL = 2, RD_KAFKA_ACL_OPERATION_READ = 3, RD_KAFKA_ACL_OPERATION_WRITE = 4, RD_KAFKA_ACL_OPERATION_CREATE = 5, RD_KAFKA_ACL_OPERATION_DELETE = 6, RD_KAFKA_ACL_OPERATION_ALTER = 7, RD_KAFKA_ACL_OPERATION_DESCRIBE = 8, RD_KAFKA_ACL_OPERATION_CLUSTER_ACTION, RD_KAFKA_ACL_OPERATION_DESCRIBE_CONFIGS, RD_KAFKA_ACL_OPERATION_ALTER_CONFIGS, RD_KAFKA_ACL_OPERATION_IDEMPOTENT_WRITE, RD_KAFKA_ACL_OPERATION__CNT } |
Apache Kafka ACL operation types. Common type for multiple Admin API functions. More... | |
typedef struct rd_kafka_AdminOptions_s | rd_kafka_AdminOptions_t |
AdminOptions provides a generic mechanism for setting optional parameters for the Admin API requests. More... | |
RD_EXPORT rd_kafka_AdminOptions_t * | rd_kafka_AdminOptions_new (rd_kafka_t *rk, rd_kafka_admin_op_t for_api) |
Create a new AdminOptions object. More... | |
RD_EXPORT void | rd_kafka_AdminOptions_destroy (rd_kafka_AdminOptions_t *options) |
Destroy a AdminOptions object. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_AdminOptions_set_request_timeout (rd_kafka_AdminOptions_t *options, int timeout_ms, char *errstr, size_t errstr_size) |
Sets the overall request timeout, including broker lookup, request transmission, operation time on broker, and response. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_AdminOptions_set_operation_timeout (rd_kafka_AdminOptions_t *options, int timeout_ms, char *errstr, size_t errstr_size) |
Sets the broker's operation timeout, such as the timeout for CreateTopics to complete the creation of topics on the controller before returning a result to the application. More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_AdminOptions_set_validate_only (rd_kafka_AdminOptions_t *options, int true_or_false, char *errstr, size_t errstr_size) |
Tell broker to only validate the request, without performing the requested operation (create topics, etc). More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_AdminOptions_set_broker (rd_kafka_AdminOptions_t *options, int32_t broker_id, char *errstr, size_t errstr_size) |
Override what broker the Admin request will be sent to. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_AdminOptions_set_require_stable_offsets (rd_kafka_AdminOptions_t *options, int true_or_false) |
Whether broker should return stable offsets (transaction-committed). More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_AdminOptions_set_include_authorized_operations (rd_kafka_AdminOptions_t *options, int true_or_false) |
Whether broker should return authorized operations for the given resource in the DescribeConsumerGroups, DescribeTopics, or DescribeCluster calls. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_AdminOptions_set_match_consumer_group_states (rd_kafka_AdminOptions_t *options, const rd_kafka_consumer_group_state_t *consumer_group_states, size_t consumer_group_states_cnt) |
Set consumer groups states to query for. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_AdminOptions_set_match_consumer_group_types (rd_kafka_AdminOptions_t *options, const rd_kafka_consumer_group_type_t *consumer_group_types, size_t consumer_group_types_cnt) |
Set consumer groups types to query for. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_AdminOptions_set_isolation_level (rd_kafka_AdminOptions_t *options, rd_kafka_IsolationLevel_t value) |
Set Isolation Level to an allowed rd_kafka_IsolationLevel_t value. | |
RD_EXPORT void | rd_kafka_AdminOptions_set_opaque (rd_kafka_AdminOptions_t *options, void *ev_opaque) |
Set application opaque value that can be extracted from the result event using rd_kafka_event_opaque() | |
Admin API - Topics | |
typedef struct rd_kafka_NewTopic_s | rd_kafka_NewTopic_t |
typedef struct rd_kafka_DeleteTopic_s | rd_kafka_DeleteTopic_t |
RD_EXPORT rd_kafka_NewTopic_t * | rd_kafka_NewTopic_new (const char *topic, int num_partitions, int replication_factor, char *errstr, size_t errstr_size) |
Create a new NewTopic object. This object is later passed to rd_kafka_CreateTopics(). More... | |
RD_EXPORT void | rd_kafka_NewTopic_destroy (rd_kafka_NewTopic_t *new_topic) |
Destroy and free a NewTopic object previously created with rd_kafka_NewTopic_new() | |
RD_EXPORT void | rd_kafka_NewTopic_destroy_array (rd_kafka_NewTopic_t **new_topics, size_t new_topic_cnt) |
Helper function to destroy all NewTopic objects in the new_topics array (of new_topic_cnt elements). The array itself is not freed. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_NewTopic_set_replica_assignment (rd_kafka_NewTopic_t *new_topic, int32_t partition, int32_t *broker_ids, size_t broker_id_cnt, char *errstr, size_t errstr_size) |
Set the replica (broker) assignment for partition to the replica set in broker_ids (of broker_id_cnt elements). More... | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_NewTopic_set_config (rd_kafka_NewTopic_t *new_topic, const char *name, const char *value) |
Set (broker-side) topic configuration name/value pair. More... | |
RD_EXPORT void | rd_kafka_CreateTopics (rd_kafka_t *rk, rd_kafka_NewTopic_t **new_topics, size_t new_topic_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Create topics in cluster as specified by the new_topics array of size new_topic_cnt elements. More... | |
const RD_EXPORT rd_kafka_topic_result_t ** | rd_kafka_CreateTopics_result_topics (const rd_kafka_CreateTopics_result_t *result, size_t *cntp) |
Get an array of topic results from a CreateTopics result. More... | |
RD_EXPORT rd_kafka_DeleteTopic_t * | rd_kafka_DeleteTopic_new (const char *topic) |
Create a new DeleteTopic object. This object is later passed to rd_kafka_DeleteTopics(). More... | |
RD_EXPORT void | rd_kafka_DeleteTopic_destroy (rd_kafka_DeleteTopic_t *del_topic) |
Destroy and free a DeleteTopic object previously created with rd_kafka_DeleteTopic_new() | |
RD_EXPORT void | rd_kafka_DeleteTopic_destroy_array (rd_kafka_DeleteTopic_t **del_topics, size_t del_topic_cnt) |
Helper function to destroy all DeleteTopic objects in the del_topics array (of del_topic_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_DeleteTopics (rd_kafka_t *rk, rd_kafka_DeleteTopic_t **del_topics, size_t del_topic_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Delete topics from cluster as specified by the topics array of size topic_cnt elements. More... | |
const RD_EXPORT rd_kafka_topic_result_t ** | rd_kafka_DeleteTopics_result_topics (const rd_kafka_DeleteTopics_result_t *result, size_t *cntp) |
Get an array of topic results from a DeleteTopics result. More... | |
Admin API - Partitions | |
typedef struct rd_kafka_NewPartitions_s | rd_kafka_NewPartitions_t |
RD_EXPORT rd_kafka_NewPartitions_t * | rd_kafka_NewPartitions_new (const char *topic, size_t new_total_cnt, char *errstr, size_t errstr_size) |
Create a new NewPartitions. This object is later passed to rd_kafka_CreatePartitions() to increase the number of partitions to new_total_cnt for an existing topic. More... | |
RD_EXPORT void | rd_kafka_NewPartitions_destroy (rd_kafka_NewPartitions_t *new_parts) |
Destroy and free a NewPartitions object previously created with rd_kafka_NewPartitions_new() | |
RD_EXPORT void | rd_kafka_NewPartitions_destroy_array (rd_kafka_NewPartitions_t **new_parts, size_t new_parts_cnt) |
Helper function to destroy all NewPartitions objects in the new_parts array (of new_parts_cnt elements). The array itself is not freed. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_NewPartitions_set_replica_assignment (rd_kafka_NewPartitions_t *new_parts, int32_t new_partition_idx, int32_t *broker_ids, size_t broker_id_cnt, char *errstr, size_t errstr_size) |
Set the replica (broker id) assignment for new_partition_idx to the replica set in broker_ids (of broker_id_cnt elements). More... | |
RD_EXPORT void | rd_kafka_CreatePartitions (rd_kafka_t *rk, rd_kafka_NewPartitions_t **new_parts, size_t new_parts_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Create additional partitions for the given topics, as specified by the new_parts array of size new_parts_cnt elements. More... | |
const RD_EXPORT rd_kafka_topic_result_t ** | rd_kafka_CreatePartitions_result_topics (const rd_kafka_CreatePartitions_result_t *result, size_t *cntp) |
Get an array of topic results from a CreatePartitions result. More... | |
Admin API - Configuration | |
enum | rd_kafka_ConfigSource_t { RD_KAFKA_CONFIG_SOURCE_UNKNOWN_CONFIG = 0, RD_KAFKA_CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG = 1, RD_KAFKA_CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG = 2, RD_KAFKA_CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG = 3, RD_KAFKA_CONFIG_SOURCE_STATIC_BROKER_CONFIG = 4, RD_KAFKA_CONFIG_SOURCE_DEFAULT_CONFIG = 5, RD_KAFKA_CONFIG_SOURCE__CNT } |
Apache Kafka config sources. More... | |
enum | rd_kafka_ResourceType_t { RD_KAFKA_RESOURCE_UNKNOWN = 0, RD_KAFKA_RESOURCE_ANY = 1, RD_KAFKA_RESOURCE_TOPIC = 2, RD_KAFKA_RESOURCE_GROUP = 3, RD_KAFKA_RESOURCE_BROKER = 4, RD_KAFKA_RESOURCE_TRANSACTIONAL_ID = 5, RD_KAFKA_RESOURCE__CNT } |
Apache Kafka resource types. More... | |
enum | rd_kafka_ResourcePatternType_t { RD_KAFKA_RESOURCE_PATTERN_UNKNOWN = 0, RD_KAFKA_RESOURCE_PATTERN_ANY = 1, RD_KAFKA_RESOURCE_PATTERN_MATCH = 2, RD_KAFKA_RESOURCE_PATTERN_LITERAL = 3, RD_KAFKA_RESOURCE_PATTERN_PREFIXED = 4, RD_KAFKA_RESOURCE_PATTERN_TYPE__CNT } |
Apache Kafka pattern types. More... | |
enum | rd_kafka_AlterConfigOpType_t { RD_KAFKA_ALTER_CONFIG_OP_TYPE_SET = 0, RD_KAFKA_ALTER_CONFIG_OP_TYPE_DELETE = 1, RD_KAFKA_ALTER_CONFIG_OP_TYPE_APPEND = 2, RD_KAFKA_ALTER_CONFIG_OP_TYPE_SUBTRACT = 3, RD_KAFKA_ALTER_CONFIG_OP_TYPE__CNT } |
Incremental alter configs operations. | |
typedef struct rd_kafka_ConfigEntry_s | rd_kafka_ConfigEntry_t |
typedef struct rd_kafka_ConfigResource_s | rd_kafka_ConfigResource_t |
const RD_EXPORT char * | rd_kafka_ConfigSource_name (rd_kafka_ConfigSource_t confsource) |
const RD_EXPORT char * | rd_kafka_ConfigEntry_name (const rd_kafka_ConfigEntry_t *entry) |
const RD_EXPORT char * | rd_kafka_ConfigEntry_value (const rd_kafka_ConfigEntry_t *entry) |
RD_EXPORT rd_kafka_ConfigSource_t | rd_kafka_ConfigEntry_source (const rd_kafka_ConfigEntry_t *entry) |
RD_EXPORT int | rd_kafka_ConfigEntry_is_read_only (const rd_kafka_ConfigEntry_t *entry) |
RD_EXPORT int | rd_kafka_ConfigEntry_is_default (const rd_kafka_ConfigEntry_t *entry) |
RD_EXPORT int | rd_kafka_ConfigEntry_is_sensitive (const rd_kafka_ConfigEntry_t *entry) |
RD_EXPORT int | rd_kafka_ConfigEntry_is_synonym (const rd_kafka_ConfigEntry_t *entry) |
const RD_EXPORT rd_kafka_ConfigEntry_t ** | rd_kafka_ConfigEntry_synonyms (const rd_kafka_ConfigEntry_t *entry, size_t *cntp) |
const RD_EXPORT char * | rd_kafka_ResourcePatternType_name (rd_kafka_ResourcePatternType_t resource_pattern_type) |
const RD_EXPORT char * | rd_kafka_ResourceType_name (rd_kafka_ResourceType_t restype) |
RD_EXPORT rd_kafka_ConfigResource_t * | rd_kafka_ConfigResource_new (rd_kafka_ResourceType_t restype, const char *resname) |
Create new ConfigResource object. More... | |
RD_EXPORT void | rd_kafka_ConfigResource_destroy (rd_kafka_ConfigResource_t *config) |
Destroy and free a ConfigResource object previously created with rd_kafka_ConfigResource_new() | |
RD_EXPORT void | rd_kafka_ConfigResource_destroy_array (rd_kafka_ConfigResource_t **config, size_t config_cnt) |
Helper function to destroy all ConfigResource objects in the configs array (of config_cnt elements). The array itself is not freed. | |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_ConfigResource_set_config (rd_kafka_ConfigResource_t *config, const char *name, const char *value) |
Set configuration name value pair. More... | |
RD_EXPORT rd_kafka_error_t * | rd_kafka_ConfigResource_add_incremental_config (rd_kafka_ConfigResource_t *config, const char *name, rd_kafka_AlterConfigOpType_t op_type, const char *value) |
Add the value of the configuration entry for a subsequent incremental alter config operation. APPEND and SUBTRACT are possible for list-type configuration entries only. More... | |
const RD_EXPORT rd_kafka_ConfigEntry_t ** | rd_kafka_ConfigResource_configs (const rd_kafka_ConfigResource_t *config, size_t *cntp) |
Get an array of config entries from a ConfigResource object. More... | |
RD_EXPORT rd_kafka_ResourceType_t | rd_kafka_ConfigResource_type (const rd_kafka_ConfigResource_t *config) |
const RD_EXPORT char * | rd_kafka_ConfigResource_name (const rd_kafka_ConfigResource_t *config) |
RD_EXPORT rd_kafka_resp_err_t | rd_kafka_ConfigResource_error (const rd_kafka_ConfigResource_t *config) |
const RD_EXPORT char * | rd_kafka_ConfigResource_error_string (const rd_kafka_ConfigResource_t *config) |
RD_EXPORT void | rd_kafka_AlterConfigs (rd_kafka_t *rk, rd_kafka_ConfigResource_t **configs, size_t config_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Update the configuration for the specified resources. Updates are not transactional so they may succeed for a subset of the provided resources while the others fail. The configuration for a particular resource is updated atomically, replacing values using the provided ConfigEntrys and reverting unspecified ConfigEntrys to their default values. More... | |
const RD_EXPORT rd_kafka_ConfigResource_t ** | rd_kafka_AlterConfigs_result_resources (const rd_kafka_AlterConfigs_result_t *result, size_t *cntp) |
Get an array of resource results from a AlterConfigs result. More... | |
RD_EXPORT void | rd_kafka_IncrementalAlterConfigs (rd_kafka_t *rk, rd_kafka_ConfigResource_t **configs, size_t config_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Incrementally update the configuration for the specified resources. Updates are not transactional so they may succeed for some resources while fail for others. The configs for a particular resource are updated atomically, executing the corresponding incremental operations on the provided configurations. More... | |
const RD_EXPORT rd_kafka_ConfigResource_t ** | rd_kafka_IncrementalAlterConfigs_result_resources (const rd_kafka_IncrementalAlterConfigs_result_t *result, size_t *cntp) |
Get an array of resource results from a IncrementalAlterConfigs result. More... | |
RD_EXPORT void | rd_kafka_DescribeConfigs (rd_kafka_t *rk, rd_kafka_ConfigResource_t **configs, size_t config_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Get configuration for the specified resources in configs . More... | |
const RD_EXPORT rd_kafka_ConfigResource_t ** | rd_kafka_DescribeConfigs_result_resources (const rd_kafka_DescribeConfigs_result_t *result, size_t *cntp) |
Get an array of resource results from a DescribeConfigs result. More... | |
Admin API - DeleteRecords | |
typedef struct rd_kafka_DeleteRecords_s | rd_kafka_DeleteRecords_t |
RD_EXPORT rd_kafka_DeleteRecords_t * | rd_kafka_DeleteRecords_new (const rd_kafka_topic_partition_list_t *before_offsets) |
Create a new DeleteRecords object. This object is later passed to rd_kafka_DeleteRecords(). More... | |
RD_EXPORT void | rd_kafka_DeleteRecords_destroy (rd_kafka_DeleteRecords_t *del_records) |
Destroy and free a DeleteRecords object previously created with rd_kafka_DeleteRecords_new() | |
RD_EXPORT void | rd_kafka_DeleteRecords_destroy_array (rd_kafka_DeleteRecords_t **del_records, size_t del_record_cnt) |
Helper function to destroy all DeleteRecords objects in the del_groups array (of del_group_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_DeleteRecords (rd_kafka_t *rk, rd_kafka_DeleteRecords_t **del_records, size_t del_record_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Delete records (messages) in topic partitions older than the offsets provided. More... | |
const RD_EXPORT rd_kafka_topic_partition_list_t * | rd_kafka_DeleteRecords_result_offsets (const rd_kafka_DeleteRecords_result_t *result) |
Get a list of topic and partition results from a DeleteRecords result. The returned objects will contain topic , partition , offset and err . offset will be set to the post-deletion low-watermark (smallest available offset of all live replicas). err will be set per-partition if deletion failed. More... | |
Admin API - DescribeTopics | |
typedef struct rd_kafka_TopicCollection_s | rd_kafka_TopicCollection_t |
Represents a collection of topics, to be passed to DescribeTopics. More... | |
typedef struct rd_kafka_TopicPartitionInfo_s | rd_kafka_TopicPartitionInfo_t |
TopicPartition represents a partition in the DescribeTopics result. More... | |
typedef struct rd_kafka_TopicDescription_s | rd_kafka_TopicDescription_t |
DescribeTopics result type. More... | |
RD_EXPORT rd_kafka_TopicCollection_t * | rd_kafka_TopicCollection_of_topic_names (const char **topics, size_t topics_cnt) |
Creates a new TopicCollection for passing to rd_kafka_DescribeTopics. More... | |
RD_EXPORT void | rd_kafka_TopicCollection_destroy (rd_kafka_TopicCollection_t *topics) |
Destroy and free a TopicCollection object created with rd_kafka_TopicCollection_new_* methods. | |
RD_EXPORT void | rd_kafka_DescribeTopics (rd_kafka_t *rk, const rd_kafka_TopicCollection_t *topics, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Describe topics as specified by the topics array of size topics_cnt elements. More... | |
const RD_EXPORT rd_kafka_TopicDescription_t ** | rd_kafka_DescribeTopics_result_topics (const rd_kafka_DescribeTopics_result_t *result, size_t *cntp) |
Get an array of topic results from a DescribeTopics result. More... | |
const RD_EXPORT rd_kafka_TopicPartitionInfo_t ** | rd_kafka_TopicDescription_partitions (const rd_kafka_TopicDescription_t *topicdesc, size_t *cntp) |
Gets an array of partitions for the topicdesc topic. More... | |
const RD_EXPORT int | rd_kafka_TopicPartitionInfo_partition (const rd_kafka_TopicPartitionInfo_t *partition) |
Gets the partition id for partition . More... | |
const RD_EXPORT rd_kafka_Node_t * | rd_kafka_TopicPartitionInfo_leader (const rd_kafka_TopicPartitionInfo_t *partition) |
Gets the partition leader for partition . More... | |
const RD_EXPORT rd_kafka_Node_t ** | rd_kafka_TopicPartitionInfo_isr (const rd_kafka_TopicPartitionInfo_t *partition, size_t *cntp) |
Gets the partition in-sync replicas for partition . More... | |
const RD_EXPORT rd_kafka_Node_t ** | rd_kafka_TopicPartitionInfo_replicas (const rd_kafka_TopicPartitionInfo_t *partition, size_t *cntp) |
Gets the partition replicas for partition . More... | |
const RD_EXPORT rd_kafka_AclOperation_t * | rd_kafka_TopicDescription_authorized_operations (const rd_kafka_TopicDescription_t *topicdesc, size_t *cntp) |
Gets the topic authorized ACL operations for the topicdesc topic. More... | |
const RD_EXPORT char * | rd_kafka_TopicDescription_name (const rd_kafka_TopicDescription_t *topicdesc) |
Gets the topic name for the topicdesc topic. More... | |
const RD_EXPORT rd_kafka_Uuid_t * | rd_kafka_TopicDescription_topic_id (const rd_kafka_TopicDescription_t *topicdesc) |
Gets the topic id for the topicdesc topic. More... | |
RD_EXPORT int | rd_kafka_TopicDescription_is_internal (const rd_kafka_TopicDescription_t *topicdesc) |
Gets if the topicdesc topic is internal. More... | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_TopicDescription_error (const rd_kafka_TopicDescription_t *topicdesc) |
Gets the error for the topicdesc topic. More... | |
Admin API - ListConsumerGroups | |
typedef struct rd_kafka_ConsumerGroupListing_s | rd_kafka_ConsumerGroupListing_t |
ListConsumerGroups result for a single group. More... | |
typedef struct rd_kafka_ListConsumerGroupsResult_s | rd_kafka_ListConsumerGroupsResult_t |
RD_EXPORT void | rd_kafka_ListConsumerGroups (rd_kafka_t *rk, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
List the consumer groups available in the cluster. More... | |
const RD_EXPORT char * | rd_kafka_ConsumerGroupListing_group_id (const rd_kafka_ConsumerGroupListing_t *grplist) |
Gets the group id for the grplist group. More... | |
RD_EXPORT int | rd_kafka_ConsumerGroupListing_is_simple_consumer_group (const rd_kafka_ConsumerGroupListing_t *grplist) |
Is the grplist group a simple consumer group. More... | |
RD_EXPORT rd_kafka_consumer_group_state_t | rd_kafka_ConsumerGroupListing_state (const rd_kafka_ConsumerGroupListing_t *grplist) |
Gets state for the grplist group. More... | |
RD_EXPORT rd_kafka_consumer_group_type_t | rd_kafka_ConsumerGroupListing_type (const rd_kafka_ConsumerGroupListing_t *grplist) |
Gets type for the grplist group. More... | |
const RD_EXPORT rd_kafka_ConsumerGroupListing_t ** | rd_kafka_ListConsumerGroups_result_valid (const rd_kafka_ListConsumerGroups_result_t *result, size_t *cntp) |
Get an array of valid list groups from a ListConsumerGroups result. More... | |
const RD_EXPORT rd_kafka_error_t ** | rd_kafka_ListConsumerGroups_result_errors (const rd_kafka_ListConsumerGroups_result_t *result, size_t *cntp) |
Get an array of errors from a ListConsumerGroups call result. More... | |
Admin API - DescribeConsumerGroups | |
typedef struct rd_kafka_ConsumerGroupDescription_s | rd_kafka_ConsumerGroupDescription_t |
DescribeConsumerGroups result type. More... | |
typedef struct rd_kafka_MemberDescription_s | rd_kafka_MemberDescription_t |
Member description included in ConsumerGroupDescription. More... | |
typedef struct rd_kafka_MemberAssignment_s | rd_kafka_MemberAssignment_t |
Member assignment included in MemberDescription. More... | |
RD_EXPORT void | rd_kafka_DescribeConsumerGroups (rd_kafka_t *rk, const char **groups, size_t groups_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Describe groups from cluster as specified by the groups array of size groups_cnt elements. More... | |
const RD_EXPORT rd_kafka_ConsumerGroupDescription_t ** | rd_kafka_DescribeConsumerGroups_result_groups (const rd_kafka_DescribeConsumerGroups_result_t *result, size_t *cntp) |
Get an array of group results from a DescribeConsumerGroups result. More... | |
const RD_EXPORT char * | rd_kafka_ConsumerGroupDescription_group_id (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets the group id for the grpdesc group. More... | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_ConsumerGroupDescription_error (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets the error for the grpdesc group. More... | |
RD_EXPORT int | rd_kafka_ConsumerGroupDescription_is_simple_consumer_group (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Is the grpdesc group a simple consumer group. More... | |
const RD_EXPORT char * | rd_kafka_ConsumerGroupDescription_partition_assignor (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets the partition assignor for the grpdesc group. More... | |
const RD_EXPORT rd_kafka_AclOperation_t * | rd_kafka_ConsumerGroupDescription_authorized_operations (const rd_kafka_ConsumerGroupDescription_t *grpdesc, size_t *cntp) |
Gets the authorized ACL operations for the grpdesc group. More... | |
RD_EXPORT rd_kafka_consumer_group_state_t | rd_kafka_ConsumerGroupDescription_state (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets state for the grpdesc group. More... | |
const RD_EXPORT rd_kafka_Node_t * | rd_kafka_ConsumerGroupDescription_coordinator (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets the coordinator for the grpdesc group. More... | |
RD_EXPORT size_t | rd_kafka_ConsumerGroupDescription_member_count (const rd_kafka_ConsumerGroupDescription_t *grpdesc) |
Gets the members count of grpdesc group. More... | |
const RD_EXPORT rd_kafka_MemberDescription_t * | rd_kafka_ConsumerGroupDescription_member (const rd_kafka_ConsumerGroupDescription_t *grpdesc, size_t idx) |
Gets a member of grpdesc group. More... | |
const RD_EXPORT char * | rd_kafka_MemberDescription_client_id (const rd_kafka_MemberDescription_t *member) |
Gets client id of member . More... | |
const RD_EXPORT char * | rd_kafka_MemberDescription_group_instance_id (const rd_kafka_MemberDescription_t *member) |
Gets group instance id of member . More... | |
const RD_EXPORT char * | rd_kafka_MemberDescription_consumer_id (const rd_kafka_MemberDescription_t *member) |
Gets consumer id of member . More... | |
const RD_EXPORT char * | rd_kafka_MemberDescription_host (const rd_kafka_MemberDescription_t *member) |
Gets host of member . More... | |
const RD_EXPORT rd_kafka_MemberAssignment_t * | rd_kafka_MemberDescription_assignment (const rd_kafka_MemberDescription_t *member) |
Gets assignment of member . More... | |
const RD_EXPORT rd_kafka_topic_partition_list_t * | rd_kafka_MemberAssignment_partitions (const rd_kafka_MemberAssignment_t *assignment) |
Gets assigned partitions of a member assignment . More... | |
Admin API - DeleteGroups | |
typedef struct rd_kafka_DeleteGroup_s | rd_kafka_DeleteGroup_t |
RD_EXPORT rd_kafka_DeleteGroup_t * | rd_kafka_DeleteGroup_new (const char *group) |
Create a new DeleteGroup object. This object is later passed to rd_kafka_DeleteGroups(). More... | |
RD_EXPORT void | rd_kafka_DeleteGroup_destroy (rd_kafka_DeleteGroup_t *del_group) |
Destroy and free a DeleteGroup object previously created with rd_kafka_DeleteGroup_new() | |
RD_EXPORT void | rd_kafka_DeleteGroup_destroy_array (rd_kafka_DeleteGroup_t **del_groups, size_t del_group_cnt) |
Helper function to destroy all DeleteGroup objects in the del_groups array (of del_group_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_DeleteGroups (rd_kafka_t *rk, rd_kafka_DeleteGroup_t **del_groups, size_t del_group_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Delete groups from cluster as specified by the del_groups array of size del_group_cnt elements. More... | |
const RD_EXPORT rd_kafka_group_result_t ** | rd_kafka_DeleteGroups_result_groups (const rd_kafka_DeleteGroups_result_t *result, size_t *cntp) |
Get an array of group results from a DeleteGroups result. More... | |
Admin API - ListConsumerGroupOffsets | |
typedef struct rd_kafka_ListConsumerGroupOffsets_s | rd_kafka_ListConsumerGroupOffsets_t |
RD_EXPORT rd_kafka_ListConsumerGroupOffsets_t * | rd_kafka_ListConsumerGroupOffsets_new (const char *group_id, const rd_kafka_topic_partition_list_t *partitions) |
Create a new ListConsumerGroupOffsets object. This object is later passed to rd_kafka_ListConsumerGroupOffsets(). More... | |
RD_EXPORT void | rd_kafka_ListConsumerGroupOffsets_destroy (rd_kafka_ListConsumerGroupOffsets_t *list_grpoffsets) |
Destroy and free a ListConsumerGroupOffsets object previously created with rd_kafka_ListConsumerGroupOffsets_new() | |
RD_EXPORT void | rd_kafka_ListConsumerGroupOffsets_destroy_array (rd_kafka_ListConsumerGroupOffsets_t **list_grpoffsets, size_t list_grpoffset_cnt) |
Helper function to destroy all ListConsumerGroupOffsets objects in the list_grpoffsets array (of list_grpoffsets_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_ListConsumerGroupOffsets (rd_kafka_t *rk, rd_kafka_ListConsumerGroupOffsets_t **list_grpoffsets, size_t list_grpoffsets_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
List committed offsets for a set of partitions in a consumer group. More... | |
const RD_EXPORT rd_kafka_group_result_t ** | rd_kafka_ListConsumerGroupOffsets_result_groups (const rd_kafka_ListConsumerGroupOffsets_result_t *result, size_t *cntp) |
Get an array of results from a ListConsumerGroupOffsets result. More... | |
Admin API - AlterConsumerGroupOffsets | |
typedef struct rd_kafka_AlterConsumerGroupOffsets_s | rd_kafka_AlterConsumerGroupOffsets_t |
RD_EXPORT rd_kafka_AlterConsumerGroupOffsets_t * | rd_kafka_AlterConsumerGroupOffsets_new (const char *group_id, const rd_kafka_topic_partition_list_t *partitions) |
Create a new AlterConsumerGroupOffsets object. This object is later passed to rd_kafka_AlterConsumerGroupOffsets(). More... | |
RD_EXPORT void | rd_kafka_AlterConsumerGroupOffsets_destroy (rd_kafka_AlterConsumerGroupOffsets_t *alter_grpoffsets) |
Destroy and free a AlterConsumerGroupOffsets object previously created with rd_kafka_AlterConsumerGroupOffsets_new() | |
RD_EXPORT void | rd_kafka_AlterConsumerGroupOffsets_destroy_array (rd_kafka_AlterConsumerGroupOffsets_t **alter_grpoffsets, size_t alter_grpoffset_cnt) |
Helper function to destroy all AlterConsumerGroupOffsets objects in the alter_grpoffsets array (of alter_grpoffsets_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_AlterConsumerGroupOffsets (rd_kafka_t *rk, rd_kafka_AlterConsumerGroupOffsets_t **alter_grpoffsets, size_t alter_grpoffsets_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Alter committed offsets for a set of partitions in a consumer group. This will succeed at the partition level only if the group is not actively subscribed to the corresponding topic. More... | |
const RD_EXPORT rd_kafka_group_result_t ** | rd_kafka_AlterConsumerGroupOffsets_result_groups (const rd_kafka_AlterConsumerGroupOffsets_result_t *result, size_t *cntp) |
Get an array of results from a AlterConsumerGroupOffsets result. More... | |
Admin API - DeleteConsumerGroupOffsets | |
typedef struct rd_kafka_DeleteConsumerGroupOffsets_s | rd_kafka_DeleteConsumerGroupOffsets_t |
RD_EXPORT rd_kafka_DeleteConsumerGroupOffsets_t * | rd_kafka_DeleteConsumerGroupOffsets_new (const char *group, const rd_kafka_topic_partition_list_t *partitions) |
Create a new DeleteConsumerGroupOffsets object. This object is later passed to rd_kafka_DeleteConsumerGroupOffsets(). More... | |
RD_EXPORT void | rd_kafka_DeleteConsumerGroupOffsets_destroy (rd_kafka_DeleteConsumerGroupOffsets_t *del_grpoffsets) |
Destroy and free a DeleteConsumerGroupOffsets object previously created with rd_kafka_DeleteConsumerGroupOffsets_new() | |
RD_EXPORT void | rd_kafka_DeleteConsumerGroupOffsets_destroy_array (rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets, size_t del_grpoffset_cnt) |
Helper function to destroy all DeleteConsumerGroupOffsets objects in the del_grpoffsets array (of del_grpoffsets_cnt elements). The array itself is not freed. | |
RD_EXPORT void | rd_kafka_DeleteConsumerGroupOffsets (rd_kafka_t *rk, rd_kafka_DeleteConsumerGroupOffsets_t **del_grpoffsets, size_t del_grpoffsets_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Delete committed offsets for a set of partitions in a consumer group. This will succeed at the partition level only if the group is not actively subscribed to the corresponding topic. More... | |
const RD_EXPORT rd_kafka_group_result_t ** | rd_kafka_DeleteConsumerGroupOffsets_result_groups (const rd_kafka_DeleteConsumerGroupOffsets_result_t *result, size_t *cntp) |
Get an array of results from a DeleteConsumerGroupOffsets result. More... | |
Admin API - ListOffsets | |
Given a topic_partition list, provides the offset information. | |
enum | rd_kafka_OffsetSpec_t { RD_KAFKA_OFFSET_SPEC_MAX_TIMESTAMP = -3, RD_KAFKA_OFFSET_SPEC_EARLIEST = -2, RD_KAFKA_OFFSET_SPEC_LATEST = -1 } |
Allows to specify the desired offsets when using ListOffsets. | |
typedef struct rd_kafka_ListOffsetsResultInfo_s | rd_kafka_ListOffsetsResultInfo_t |
Information returned from a ListOffsets call for a specific rd_kafka_topic_partition_t . | |
const RD_EXPORT rd_kafka_topic_partition_t * | rd_kafka_ListOffsetsResultInfo_topic_partition (const rd_kafka_ListOffsetsResultInfo_t *result_info) |
Returns the topic partition of the passed result_info . | |
RD_EXPORT int64_t | rd_kafka_ListOffsetsResultInfo_timestamp (const rd_kafka_ListOffsetsResultInfo_t *result_info) |
Returns the timestamp corresponding to the offset in result_info . | |
const RD_EXPORT rd_kafka_ListOffsetsResultInfo_t ** | rd_kafka_ListOffsets_result_infos (const rd_kafka_ListOffsets_result_t *result, size_t *cntp) |
Returns the array of ListOffsetsResultInfo in result and populates the size of the array in cntp . | |
RD_EXPORT void | rd_kafka_ListOffsets (rd_kafka_t *rk, rd_kafka_topic_partition_list_t *topic_partitions, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
List offsets for the specified topic_partitions . This operation enables to find the beginning offset, end offset as well as the offset matching a timestamp in partitions or the offset with max timestamp. More... | |
Admin API - User SCRAM credentials | |
enum | rd_kafka_ScramMechanism_t { RD_KAFKA_SCRAM_MECHANISM_UNKNOWN = 0, RD_KAFKA_SCRAM_MECHANISM_SHA_256 = 1, RD_KAFKA_SCRAM_MECHANISM_SHA_512 = 2, RD_KAFKA_SCRAM_MECHANISM__CNT } |
Apache Kafka ScramMechanism values. | |
typedef struct rd_kafka_ScramCredentialInfo_s | rd_kafka_ScramCredentialInfo_t |
Scram credential info. Mechanism and iterations for a SASL/SCRAM credential associated with a user. | |
typedef struct rd_kafka_UserScramCredentialsDescription_s | rd_kafka_UserScramCredentialsDescription_t |
Representation of all SASL/SCRAM credentials associated with a user that can be retrieved, or an error indicating why credentials could not be retrieved. | |
typedef struct rd_kafka_UserScramCredentialAlteration_s | rd_kafka_UserScramCredentialAlteration_t |
A request to alter a user's SASL/SCRAM credentials. | |
typedef struct rd_kafka_AlterUserScramCredentials_result_response_s | rd_kafka_AlterUserScramCredentials_result_response_t |
Result of a single user SCRAM alteration. | |
RD_EXPORT rd_kafka_ScramMechanism_t | rd_kafka_ScramCredentialInfo_mechanism (const rd_kafka_ScramCredentialInfo_t *scram_credential_info) |
Returns the mechanism of a given ScramCredentialInfo. | |
RD_EXPORT int32_t | rd_kafka_ScramCredentialInfo_iterations (const rd_kafka_ScramCredentialInfo_t *scram_credential_info) |
Returns the iterations of a given ScramCredentialInfo. | |
const RD_EXPORT char * | rd_kafka_UserScramCredentialsDescription_user (const rd_kafka_UserScramCredentialsDescription_t *description) |
Returns the username of a UserScramCredentialsDescription. | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_UserScramCredentialsDescription_error (const rd_kafka_UserScramCredentialsDescription_t *description) |
Returns the error associated with a UserScramCredentialsDescription. | |
RD_EXPORT size_t | rd_kafka_UserScramCredentialsDescription_scramcredentialinfo_count (const rd_kafka_UserScramCredentialsDescription_t *description) |
Returns the count of ScramCredentialInfos of a UserScramCredentialsDescription. | |
const RD_EXPORT rd_kafka_ScramCredentialInfo_t * | rd_kafka_UserScramCredentialsDescription_scramcredentialinfo (const rd_kafka_UserScramCredentialsDescription_t *description, size_t idx) |
Returns the ScramCredentialInfo at index idx of UserScramCredentialsDescription. | |
const RD_EXPORT rd_kafka_UserScramCredentialsDescription_t ** | rd_kafka_DescribeUserScramCredentials_result_descriptions (const rd_kafka_DescribeUserScramCredentials_result_t *result, size_t *cntp) |
Get an array of descriptions from a DescribeUserScramCredentials result. More... | |
RD_EXPORT void | rd_kafka_DescribeUserScramCredentials (rd_kafka_t *rk, const char **users, size_t user_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Describe SASL/SCRAM credentials. This operation is supported by brokers with version 2.7.0 or higher. More... | |
RD_EXPORT rd_kafka_UserScramCredentialAlteration_t * | rd_kafka_UserScramCredentialUpsertion_new (const char *username, rd_kafka_ScramMechanism_t mechanism, int32_t iterations, const unsigned char *password, size_t password_size, const unsigned char *salt, size_t salt_size) |
Allocates a new UserScramCredentialUpsertion given its fields. If salt isn't given a 64 B salt is generated using OpenSSL RAND_priv_bytes, if available. More... | |
RD_EXPORT rd_kafka_UserScramCredentialAlteration_t * | rd_kafka_UserScramCredentialDeletion_new (const char *username, rd_kafka_ScramMechanism_t mechanism) |
Allocates a new UserScramCredentialDeletion given its fields. More... | |
RD_EXPORT void | rd_kafka_UserScramCredentialAlteration_destroy (rd_kafka_UserScramCredentialAlteration_t *alteration) |
Destroys a UserScramCredentialAlteration given its pointer. | |
RD_EXPORT void | rd_kafka_UserScramCredentialAlteration_destroy_array (rd_kafka_UserScramCredentialAlteration_t **alterations, size_t alteration_cnt) |
Destroys an array of UserScramCredentialAlteration. | |
const RD_EXPORT char * | rd_kafka_AlterUserScramCredentials_result_response_user (const rd_kafka_AlterUserScramCredentials_result_response_t *response) |
Returns the username for a rd_kafka_AlterUserScramCredentials_result_response. | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_AlterUserScramCredentials_result_response_error (const rd_kafka_AlterUserScramCredentials_result_response_t *response) |
Returns the error of a rd_kafka_AlterUserScramCredentials_result_response. | |
const RD_EXPORT rd_kafka_AlterUserScramCredentials_result_response_t ** | rd_kafka_AlterUserScramCredentials_result_responses (const rd_kafka_AlterUserScramCredentials_result_t *result, size_t *cntp) |
Get an array of responses from a AlterUserScramCredentials result. More... | |
RD_EXPORT void | rd_kafka_AlterUserScramCredentials (rd_kafka_t *rk, rd_kafka_UserScramCredentialAlteration_t **alterations, size_t alteration_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Alter SASL/SCRAM credentials. This operation is supported by brokers with version 2.7.0 or higher. More... | |
Admin API - ACL operations | |
enum | rd_kafka_AclPermissionType_t { RD_KAFKA_ACL_PERMISSION_TYPE_UNKNOWN = 0, RD_KAFKA_ACL_PERMISSION_TYPE_ANY, RD_KAFKA_ACL_PERMISSION_TYPE_DENY = 2, RD_KAFKA_ACL_PERMISSION_TYPE_ALLOW = 3, RD_KAFKA_ACL_PERMISSION_TYPE__CNT } |
Apache Kafka ACL permission types. More... | |
typedef struct rd_kafka_AclBinding_s | rd_kafka_AclBinding_t |
ACL Binding is used to create access control lists. More... | |
typedef rd_kafka_AclBinding_t | rd_kafka_AclBindingFilter_t |
ACL Binding filter is used to filter access control lists. More... | |
typedef struct rd_kafka_DeleteAcls_result_response_s | rd_kafka_DeleteAcls_result_response_t |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_acl_result_error (const rd_kafka_acl_result_t *aclres) |
const RD_EXPORT char * | rd_kafka_AclOperation_name (rd_kafka_AclOperation_t acl_operation) |
const RD_EXPORT char * | rd_kafka_AclPermissionType_name (rd_kafka_AclPermissionType_t acl_permission_type) |
RD_EXPORT rd_kafka_AclBinding_t * | rd_kafka_AclBinding_new (rd_kafka_ResourceType_t restype, const char *name, rd_kafka_ResourcePatternType_t resource_pattern_type, const char *principal, const char *host, rd_kafka_AclOperation_t operation, rd_kafka_AclPermissionType_t permission_type, char *errstr, size_t errstr_size) |
Create a new AclBinding object. This object is later passed to rd_kafka_CreateAcls(). More... | |
RD_EXPORT rd_kafka_AclBindingFilter_t * | rd_kafka_AclBindingFilter_new (rd_kafka_ResourceType_t restype, const char *name, rd_kafka_ResourcePatternType_t resource_pattern_type, const char *principal, const char *host, rd_kafka_AclOperation_t operation, rd_kafka_AclPermissionType_t permission_type, char *errstr, size_t errstr_size) |
Create a new AclBindingFilter object. This object is later passed to rd_kafka_DescribeAcls() or rd_kafka_DeletesAcls() in order to filter the acls to retrieve or to delete. Use the same rd_kafka_AclBinding functions to query or destroy it. More... | |
RD_EXPORT rd_kafka_ResourceType_t | rd_kafka_AclBinding_restype (const rd_kafka_AclBinding_t *acl) |
const RD_EXPORT char * | rd_kafka_AclBinding_name (const rd_kafka_AclBinding_t *acl) |
const RD_EXPORT char * | rd_kafka_AclBinding_principal (const rd_kafka_AclBinding_t *acl) |
const RD_EXPORT char * | rd_kafka_AclBinding_host (const rd_kafka_AclBinding_t *acl) |
RD_EXPORT rd_kafka_AclOperation_t | rd_kafka_AclBinding_operation (const rd_kafka_AclBinding_t *acl) |
RD_EXPORT rd_kafka_AclPermissionType_t | rd_kafka_AclBinding_permission_type (const rd_kafka_AclBinding_t *acl) |
RD_EXPORT rd_kafka_ResourcePatternType_t | rd_kafka_AclBinding_resource_pattern_type (const rd_kafka_AclBinding_t *acl) |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_AclBinding_error (const rd_kafka_AclBinding_t *acl) |
RD_EXPORT void | rd_kafka_AclBinding_destroy (rd_kafka_AclBinding_t *acl_binding) |
Destroy and free an AclBinding object previously created with rd_kafka_AclBinding_new() | |
RD_EXPORT void | rd_kafka_AclBinding_destroy_array (rd_kafka_AclBinding_t **acl_bindings, size_t acl_bindings_cnt) |
Helper function to destroy all AclBinding objects in the acl_bindings array (of acl_bindings_cnt elements). The array itself is not freed. | |
const RD_EXPORT rd_kafka_acl_result_t ** | rd_kafka_CreateAcls_result_acls (const rd_kafka_CreateAcls_result_t *result, size_t *cntp) |
Get an array of acl results from a CreateAcls result. More... | |
RD_EXPORT void | rd_kafka_CreateAcls (rd_kafka_t *rk, rd_kafka_AclBinding_t **new_acls, size_t new_acls_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Create acls as specified by the new_acls array of size new_topic_cnt elements. More... | |
const RD_EXPORT rd_kafka_AclBinding_t ** | rd_kafka_DescribeAcls_result_acls (const rd_kafka_DescribeAcls_result_t *result, size_t *cntp) |
Get an array of resource results from a DescribeAcls result. More... | |
RD_EXPORT void | rd_kafka_DescribeAcls (rd_kafka_t *rk, rd_kafka_AclBindingFilter_t *acl_filter, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Describe acls matching the filter provided in acl_filter . More... | |
const RD_EXPORT rd_kafka_DeleteAcls_result_response_t ** | rd_kafka_DeleteAcls_result_responses (const rd_kafka_DeleteAcls_result_t *result, size_t *cntp) |
Get an array of DeleteAcls result responses from a DeleteAcls result. More... | |
const RD_EXPORT rd_kafka_error_t * | rd_kafka_DeleteAcls_result_response_error (const rd_kafka_DeleteAcls_result_response_t *result_response) |
const RD_EXPORT rd_kafka_AclBinding_t ** | rd_kafka_DeleteAcls_result_response_matching_acls (const rd_kafka_DeleteAcls_result_response_t *result_response, size_t *matching_acls_cntp) |
RD_EXPORT void | rd_kafka_DeleteAcls (rd_kafka_t *rk, rd_kafka_AclBindingFilter_t **del_acls, size_t del_acls_cnt, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Delete acls matching the filteres provided in del_acls array of size del_acls_cnt . More... | |
Admin API - Elect Leaders | |
enum | rd_kafka_ElectionType_t { RD_KAFKA_ELECTION_TYPE_PREFERRED = 0, RD_KAFKA_ELECTION_TYPE_UNCLEAN = 1 } |
Apache Kafka Election Types. More... | |
typedef struct rd_kafka_ElectLeaders_s | rd_kafka_ElectLeaders_t |
Represents elect leaders request. | |
RD_EXPORT rd_kafka_ElectLeaders_t * | rd_kafka_ElectLeaders_new (rd_kafka_ElectionType_t election_type, rd_kafka_topic_partition_list_t *partitions) |
Create a new rd_kafka_ElectLeaders_t object. This object is later passed to rd_kafka_ElectLeaders(). More... | |
RD_EXPORT void | rd_kafka_ElectLeaders_destroy (rd_kafka_ElectLeaders_t *elect_leaders) |
Destroy and free a rd_kafka_ElectLeaders_t object previously created with rd_kafka_ElectLeaders_new() More... | |
RD_EXPORT void | rd_kafka_ElectLeaders (rd_kafka_t *rk, rd_kafka_ElectLeaders_t *elect_leaders, const rd_kafka_AdminOptions_t *options, rd_kafka_queue_t *rkqu) |
Elect Leaders for the provided Topic Partitions according to the specified election type. More... | |
const RD_EXPORT rd_kafka_topic_partition_result_t ** | rd_kafka_ElectLeaders_result_partitions (const rd_kafka_ElectLeaders_result_t *result, size_t *cntp) |
Get the array of topic partition result objects from the elect leaders result event and populates the size of the array in cntp . More... | |
Apache Kafka C/C++ consumer and producer client library.
rdkafka.h contains the public API for librdkafka. The API is documented in this file as comments prefixing the function, type, enum, define, etc.
#define RD_KAFKA_VERSION 0x020600ff |
librdkafka version
Interpreted as hex MM.mm.rr.xx
:
E.g.: 0x000801ff
= 0.8.1
#define RD_KAFKA_DEBUG_CONTEXTS |
Supported debug contexts. (compile time)
#define RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION |
Not leader for partition
#define RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS |
Group coordinator load in progress
#define RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE |
Group coordinator not available
#define RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP RD_KAFKA_RESP_ERR_NOT_COORDINATOR |
Not coordinator for group
#define RD_KAFKA_V_END RD_KAFKA_VTYPE_END |
Convenience macros for rd_kafka_vtype_t that takes the correct arguments for each vtype.
va-arg end sentinel used to terminate the variable argument list
#define RD_KAFKA_V_TOPIC | ( | topic | ) |
Topic name (const char *)
rd_kafka_vu_t field: u.cstr
#define RD_KAFKA_V_RKT | ( | rkt | ) |
Topic object (rd_kafka_topic_t *)
rd_kafka_vu_t field: u.rkt
#define RD_KAFKA_V_PARTITION | ( | partition | ) |
Partition (int32_t)
rd_kafka_vu_t field: u.i32
#define RD_KAFKA_V_VALUE | ( | VALUE, | |
LEN | |||
) |
Message value/payload pointer and length (void *, size_t)
rd_kafka_vu_t fields: u.mem.ptr, u.mem.size
#define RD_KAFKA_V_KEY | ( | KEY, | |
LEN | |||
) |
Message key pointer and length (const void *, size_t)
rd_kafka_vu_t field: u.mem.ptr, rd_kafka_vu.t.u.mem.size
#define RD_KAFKA_V_OPAQUE | ( | msg_opaque | ) |
Message opaque pointer (void *) Same as msg_opaque
, produce
(.., msg_opaque), and rkmessage->_private
.
rd_kafka_vu_t field: u.ptr
#define RD_KAFKA_V_MSGFLAGS | ( | msgflags | ) | _LRK_TYPECHECK(RD_KAFKA_VTYPE_MSGFLAGS, int, msgflags), (int)msgflags |
#define RD_KAFKA_V_TIMESTAMP | ( | timestamp | ) |
Timestamp in milliseconds since epoch UTC (int64_t). A value of 0 will use the current wall-clock time.
rd_kafka_vu_t field: u.i64
#define RD_KAFKA_V_HEADER | ( | NAME, | |
VALUE, | |||
LEN | |||
) |
Add Message Header (const char *NAME, const void *VALUE, ssize_t LEN).
rd_kafka_vu_t fields: u.header.name, u.header.val, u.header.size
#define RD_KAFKA_V_HEADERS | ( | HDRS | ) |
Message Headers list (rd_kafka_headers_t *). The message object will assume ownership of the headers (unless producev() fails). Any existing headers will be replaced.
rd_kafka_vu_t fields: u.headers
#define RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE 0x8 |
Flags for rd_kafka_destroy_flags()
Don't call consumer_close() to leave group and commit final offsets.
This also disables consumer callbacks to be called from rd_kafka_destroy*(), such as rebalance_cb.
The consumer group handler is still closed internally, but from an application perspective none of the functionality from consumer_close() is performed.
#define RD_KAFKA_PARTITION_UA ((int32_t)-1) |
Unassigned partition.
The unassigned partition is used by the producer API for messages that should be partitioned using the configured or default partitioner.
#define RD_KAFKA_OFFSET_BEGINNING -2 |
Start consuming from beginning of \ kafka partition queue: oldest msg
#define RD_KAFKA_OFFSET_END -1 |
Start consuming from end of kafka \ partition queue: next msg
#define RD_KAFKA_OFFSET_STORED -1000 |
Start consuming from offset retrieved \ from offset store
#define RD_KAFKA_OFFSET_INVALID -1001 |
Invalid offset
#define RD_KAFKA_OFFSET_TAIL | ( | CNT | ) | (RD_KAFKA_OFFSET_TAIL_BASE - (CNT)) |
Start consuming CNT
messages from topic's current end offset.
That is, if current end offset is 12345 and CNT
is 200, it will start consuming from offset 12345-200
= 12145
.
#define RD_KAFKA_MSG_F_FREE 0x1 |
Producer message flags.
Delegate freeing of payload to rdkafka. \
#define RD_KAFKA_MSG_F_COPY 0x2 |
rdkafka will make a copy of the payload. \
#define RD_KAFKA_MSG_F_BLOCK 0x4 |
Block produce*() on message queue full. \ WARNING: If a delivery report callback \ is used, the application MUST \ call rd_kafka_poll() (or equiv.) \ to make sure delivered messages \ are drained from the internal \ delivery report queue. \ Failure to do so will result \ in indefinitely blocking on \ the produce() call when the \ message queue is full.
#define RD_KAFKA_MSG_F_PARTITION 0x8 |
produce_batch() will honor \ per-message partition.
#define RD_KAFKA_PURGE_F_QUEUE 0x1 |
Flags for rd_kafka_purge()
Purge messages in internal queues.
#define RD_KAFKA_PURGE_F_INFLIGHT 0x2 |
Purge messages in-flight to or from the broker. Purging these messages will void any future acknowledgements from the broker, making it impossible for the application to know if these messages were successfully delivered or not. Retrying these messages may lead to duplicates.
#define RD_KAFKA_PURGE_F_NON_BLOCKING 0x4 |
Don't wait for background thread queue purging to finish.
#define RD_KAFKA_EVENT_NONE 0x0 |
Unset value
#define RD_KAFKA_EVENT_DR 0x1 |
Producer Delivery report batch
#define RD_KAFKA_EVENT_FETCH 0x2 |
Fetched message (consumer)
#define RD_KAFKA_EVENT_LOG 0x4 |
Log message
#define RD_KAFKA_EVENT_ERROR 0x8 |
Error
#define RD_KAFKA_EVENT_REBALANCE 0x10 |
Group rebalance (consumer)
#define RD_KAFKA_EVENT_OFFSET_COMMIT 0x20 |
Offset commit result
#define RD_KAFKA_EVENT_STATS 0x40 |
Stats
#define RD_KAFKA_EVENT_CREATETOPICS_RESULT 100 |
CreateTopics_result_t
#define RD_KAFKA_EVENT_DELETETOPICS_RESULT 101 |
DeleteTopics_result_t
#define RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT 102 |
CreatePartitions_result_t
#define RD_KAFKA_EVENT_ALTERCONFIGS_RESULT 103 |
AlterConfigs_result_t
#define RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT 104 |
DescribeConfigs_result_t
#define RD_KAFKA_EVENT_DELETERECORDS_RESULT 105 |
DeleteRecords_result_t
#define RD_KAFKA_EVENT_DELETEGROUPS_RESULT 106 |
DeleteGroups_result_t
#define RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT 107 |
DeleteConsumerGroupOffsets_result_t
#define RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH 0x100 |
SASL/OAUTHBEARER token needs to be refreshed
#define RD_KAFKA_EVENT_BACKGROUND 0x200 |
Enable background thread.
#define RD_KAFKA_EVENT_CREATEACLS_RESULT 0x400 |
CreateAcls_result_t
#define RD_KAFKA_EVENT_DESCRIBEACLS_RESULT 0x800 |
DescribeAcls_result_t
#define RD_KAFKA_EVENT_DELETEACLS_RESULT 0x1000 |
DeleteAcls_result_t
#define RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT 0x2000 |
ListConsumerGroupsResult_t
#define RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT 0x4000 |
DescribeConsumerGroups_result_t
#define RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT 0x8000 |
ListConsumerGroupOffsets_result_t
#define RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT 0x10000 |
AlterConsumerGroupOffsets_result_t
#define RD_KAFKA_EVENT_INCREMENTALALTERCONFIGS_RESULT 0x20000 |
IncrementalAlterConfigs_result_t
#define RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT 0x40000 |
DescribeUserScramCredentials_result_t
#define RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT 0x80000 |
AlterUserScramCredentials_result_t
#define RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT 0x100000 |
DescribeTopics_result_t
#define RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT 0x200000 |
DescribeCluster_result_t
#define RD_KAFKA_EVENT_LISTOFFSETS_RESULT 0x400000 |
ListOffsets_result_t
#define RD_KAFKA_EVENT_ELECTLEADERS_RESULT 0x800000 |
ElectLeaders_result_t
typedef rd_kafka_event_t rd_kafka_CreateTopics_result_t |
CreateTopics result type
typedef rd_kafka_event_t rd_kafka_DeleteTopics_result_t |
DeleteTopics result type
typedef rd_kafka_event_t rd_kafka_CreateAcls_result_t |
CreateAcls result type
typedef rd_kafka_event_t rd_kafka_DescribeAcls_result_t |
DescribeAcls result type
typedef rd_kafka_event_t rd_kafka_DeleteAcls_result_t |
DeleteAcls result type
typedef rd_kafka_event_t rd_kafka_CreatePartitions_result_t |
CreatePartitions result type
typedef rd_kafka_event_t rd_kafka_AlterConfigs_result_t |
AlterConfigs result type
typedef rd_kafka_event_t rd_kafka_IncrementalAlterConfigs_result_t |
IncrementalAlterConfigs result type
typedef rd_kafka_event_t rd_kafka_DescribeConfigs_result_t |
CreateTopics result type
typedef rd_kafka_event_t rd_kafka_DeleteRecords_result_t |
DeleteRecords result type
typedef rd_kafka_event_t rd_kafka_ListConsumerGroups_result_t |
ListConsumerGroups result type
typedef rd_kafka_event_t rd_kafka_DescribeConsumerGroups_result_t |
DescribeConsumerGroups result type
typedef rd_kafka_event_t rd_kafka_DeleteGroups_result_t |
DeleteGroups result type
typedef rd_kafka_event_t rd_kafka_DeleteConsumerGroupOffsets_result_t |
DeleteConsumerGroupOffsets result type
typedef rd_kafka_event_t rd_kafka_AlterConsumerGroupOffsets_result_t |
AlterConsumerGroupOffsets result type
typedef rd_kafka_event_t rd_kafka_ListConsumerGroupOffsets_result_t |
ListConsumerGroupOffsets result type
typedef rd_kafka_event_t rd_kafka_DescribeTopics_result_t |
DescribeTopics result type
typedef rd_kafka_event_t rd_kafka_DescribeCluster_result_t |
DescribeCluster result type
typedef rd_kafka_event_t rd_kafka_DescribeUserScramCredentials_result_t |
DescribeUserScramCredentials result type
typedef rd_kafka_event_t rd_kafka_AlterUserScramCredentials_result_t |
AlterUserScramCredentials result type
typedef rd_kafka_event_t rd_kafka_ListOffsets_result_t |
ListOffsets result type
typedef rd_kafka_event_t rd_kafka_ElectLeaders_result_t |
ElectLeaders result type
typedef rd_kafka_resp_err_t() rd_kafka_plugin_f_conf_init_t(rd_kafka_conf_t *conf, void **plug_opaquep, char *errstr, size_t errstr_size) |
Plugin's configuration initializer method called each time the library is referenced from configuration (even if previously loaded by another client instance).
conf_init
conf | Configuration set up to this point. |
plug_opaquep | Plugin can set this pointer to a per-configuration opaque pointer. |
errstr | String buffer of size errstr_size where plugin must write a human readable error string in the case the initializer fails (returns non-zero). |
errstr_size | Maximum space (including \0) in errstr . |
typedef rd_kafka_conf_res_t() rd_kafka_interceptor_f_on_conf_set_t(rd_kafka_conf_t *conf, const char *name, const char *val, char *errstr, size_t errstr_size, void *ic_opaque) |
on_conf_set() is called from rd_kafka_*_conf_set() in the order the interceptors were added.
conf | Configuration object. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
name | The configuration property to set. |
val | The configuration value to set, or NULL for reverting to default in which case the previous value should be freed. |
errstr | A human readable error string in case the interceptor fails. |
errstr_size | Maximum space (including \0) in errstr . |
typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_conf_dup_t(rd_kafka_conf_t *new_conf, const rd_kafka_conf_t *old_conf, size_t filter_cnt, const char **filter, void *ic_opaque) |
on_conf_dup() is called from rd_kafka_conf_dup() in the order the interceptors were added and is used to let an interceptor re-register its conf interecptors with a new opaque value. The on_conf_dup() method is called prior to the configuration from old_conf
being copied to new_conf
.
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
new_conf | New configuration object. |
old_conf | Old configuration object to copy properties from. |
filter_cnt | Number of property names to filter in filter . |
filter | Property names to filter out (ignore) when setting up new_conf . |
typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_conf_destroy_t(void *ic_opaque) |
on_conf_destroy() is called from rd_kafka_*_conf_destroy() in the order the interceptors were added.
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_new_t(rd_kafka_t *rk, const rd_kafka_conf_t *conf, void *ic_opaque, char *errstr, size_t errstr_size) |
on_new() is called from rd_kafka_new() prior toreturning the newly created client instance to the application.
rk | The client instance. |
conf | The client instance's final configuration. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
errstr | A human readable error string in case the interceptor fails. |
errstr_size | Maximum space (including \0) in errstr . |
rk
client instance will not be fully set up when this interceptor is called and the interceptor MUST NOT call any other rk-specific APIs than rd_kafka_interceptor_add..(). typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_destroy_t(rd_kafka_t *rk, void *ic_opaque) |
on_destroy() is called from rd_kafka_destroy() or (rd_kafka_new() if rd_kafka_new() fails during initialization).
rk | The client instance. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_send_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_send() is called from rd_kafka_produce*() (et.al) prior to the partitioner being called.
rk | The client instance. |
rkmessage | The message being produced. Immutable. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rkmessage
object is NOT mutable and MUST NOT be modified by the interceptor.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_acknowledgement_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_acknowledgement() is called to inform interceptors that a message was succesfully delivered or permanently failed delivery. The interceptor chain is called from internal librdkafka background threads, or rd_kafka_produce*() if the partitioner failed.
rk | The client instance. |
rkmessage | The message being produced. Immutable. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rkmessage
object is NOT mutable and MUST NOT be modified by the interceptor.rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_consume_t(rd_kafka_t *rk, rd_kafka_message_t *rkmessage, void *ic_opaque) |
on_consume() is called just prior to passing the message to the application in rd_kafka_consumer_poll(), rd_kafka_consume*(), the event interface, etc.
rk | The client instance. |
rkmessage | The message being consumed. Immutable. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rkmessage
object is NOT mutable and MUST NOT be modified by the interceptor.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_commit_t(rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_resp_err_t err, void *ic_opaque) |
on_commit() is called on completed or failed offset commit. It is called from internal librdkafka threads.
rk | The client instance. |
offsets | List of topic+partition+offset+error that were committed. The error message of each partition should be checked for error. |
err | The commit error, if any. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_request_sent_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, void *ic_opaque) |
on_request_sent() is called when a request has been fully written to a broker TCP connections socket.
rk | The client instance. |
sockfd | Socket file descriptor. |
brokername | Broker request is being sent to. |
brokerid | Broker request is being sent to. |
ApiKey | Kafka protocol request type. |
ApiVersion | Kafka protocol request type version. |
CorrId | Kafka protocol request correlation id. |
size | Size of request. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_response_received_t(rd_kafka_t *rk, int sockfd, const char *brokername, int32_t brokerid, int16_t ApiKey, int16_t ApiVersion, int32_t CorrId, size_t size, int64_t rtt, rd_kafka_resp_err_t err, void *ic_opaque) |
on_response_received() is called when a protocol response has been fully received from a broker TCP connection socket but before the response payload is parsed.
rk | The client instance. |
sockfd | Socket file descriptor (always -1). |
brokername | Broker response was received from, possibly empty string on error. |
brokerid | Broker response was received from. |
ApiKey | Kafka protocol request type or -1 on error. |
ApiVersion | Kafka protocol request type version or -1 on error. |
CorrId | Kafka protocol request correlation id, possibly -1 on error. |
size | Size of response, possibly 0 on error. |
rtt | Request round-trip-time in microseconds, possibly -1 on error. |
err | Receive error. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_thread_start_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque) |
on_thread_start() is called from a newly created librdkafka-managed thread.
rk | The client instance. |
thread_type | Thread type. |
thread_name | Human-readable thread name, may not be unique. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_thread_exit_t(rd_kafka_t *rk, rd_kafka_thread_type_t thread_type, const char *thread_name, void *ic_opaque) |
on_thread_exit() is called just prior to a librdkafka-managed thread exiting from the exiting thread itself.
rk | The client instance. |
thread_type | Thread type.n |
thread_name | Human-readable thread name, may not be unique. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
rk
, or perform any blocking or prolonged work.typedef rd_kafka_resp_err_t() rd_kafka_interceptor_f_on_broker_state_change_t(rd_kafka_t *rk, int32_t broker_id, const char *secproto, const char *name, int port, const char *state, void *ic_opaque) |
on_broker_state_change() is called just after a broker has been created or its state has been changed.
rk | The client instance. |
broker_id | The broker id (-1 is used for bootstrap brokers). |
secproto | The security protocol. |
name | The original name of the broker. |
port | The port of the broker. |
state | Broker state name. |
ic_opaque | The interceptor's opaque pointer specified in ..add..(). |
typedef struct rd_kafka_AdminOptions_s rd_kafka_AdminOptions_t |
AdminOptions provides a generic mechanism for setting optional parameters for the Admin API requests.
typedef struct rd_kafka_NewTopic_s rd_kafka_NewTopic_t |
Defines a new topic to be created.
typedef struct rd_kafka_DeleteTopic_s rd_kafka_DeleteTopic_t |
Represents a topic to be deleted.
typedef struct rd_kafka_NewPartitions_s rd_kafka_NewPartitions_t |
Defines a new partition to be created.
typedef struct rd_kafka_ConfigEntry_s rd_kafka_ConfigEntry_t |
Apache Kafka configuration entry.
typedef struct rd_kafka_ConfigResource_s rd_kafka_ConfigResource_t |
Apache Kafka configuration resource.
typedef struct rd_kafka_DeleteRecords_s rd_kafka_DeleteRecords_t |
! Represents records to be deleted
typedef struct rd_kafka_TopicCollection_s rd_kafka_TopicCollection_t |
Represents a collection of topics, to be passed to DescribeTopics.
typedef struct rd_kafka_TopicPartitionInfo_s rd_kafka_TopicPartitionInfo_t |
TopicPartition represents a partition in the DescribeTopics result.
typedef struct rd_kafka_TopicDescription_s rd_kafka_TopicDescription_t |
DescribeTopics result type.
typedef struct rd_kafka_ConsumerGroupListing_s rd_kafka_ConsumerGroupListing_t |
ListConsumerGroups result for a single group.
! ListConsumerGroups result for a single group
typedef struct rd_kafka_ListConsumerGroupsResult_s rd_kafka_ListConsumerGroupsResult_t |
! ListConsumerGroups results and errors
typedef struct rd_kafka_ConsumerGroupDescription_s rd_kafka_ConsumerGroupDescription_t |
DescribeConsumerGroups result type.
typedef struct rd_kafka_MemberDescription_s rd_kafka_MemberDescription_t |
Member description included in ConsumerGroupDescription.
typedef struct rd_kafka_MemberAssignment_s rd_kafka_MemberAssignment_t |
Member assignment included in MemberDescription.
typedef struct rd_kafka_DeleteGroup_s rd_kafka_DeleteGroup_t |
Represents a group to be deleted.
typedef struct rd_kafka_ListConsumerGroupOffsets_s rd_kafka_ListConsumerGroupOffsets_t |
Represents consumer group committed offsets to be listed.
typedef struct rd_kafka_AlterConsumerGroupOffsets_s rd_kafka_AlterConsumerGroupOffsets_t |
Represents consumer group committed offsets to be altered.
typedef struct rd_kafka_DeleteConsumerGroupOffsets_s rd_kafka_DeleteConsumerGroupOffsets_t |
Represents consumer group committed offsets to be deleted.
typedef struct rd_kafka_AclBinding_s rd_kafka_AclBinding_t |
ACL Binding is used to create access control lists.
ACL Binding filter is used to filter access control lists.
typedef struct rd_kafka_DeleteAcls_result_response_s rd_kafka_DeleteAcls_result_response_t |
DeleteAcls - delete access control lists.
enum rd_kafka_type_t |
rd_kafka_t handle type.
Enumerator | |
---|---|
RD_KAFKA_PRODUCER | Producer client |
RD_KAFKA_CONSUMER | Consumer client |
Timestamp types
Enumerator | |
---|---|
RD_KAFKA_TIMESTAMP_NOT_AVAILABLE | Timestamp not available |
RD_KAFKA_TIMESTAMP_CREATE_TIME | Message creation time |
RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME | Log append time |
enum rd_kafka_resp_err_t |
Error codes.
The negative error codes delimited by two underscores (RD_KAFKA_RESP_ERR__
..) denotes errors internal to librdkafka and are displayed as "Local:
<error string..>", while the error codes delimited by a single underscore (RD_KAFKA_RESP_ERR_
..) denote broker errors and are displayed as "Broker:
<error string..>".
Enumerator | |
---|---|
RD_KAFKA_RESP_ERR__BEGIN | Begin internal error codes |
RD_KAFKA_RESP_ERR__BAD_MSG | Received message is incorrect |
RD_KAFKA_RESP_ERR__BAD_COMPRESSION | Bad/unknown compression |
RD_KAFKA_RESP_ERR__DESTROY | Broker is going away |
RD_KAFKA_RESP_ERR__FAIL | Generic failure |
RD_KAFKA_RESP_ERR__TRANSPORT | Broker transport failure |
RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE | Critical system resource |
RD_KAFKA_RESP_ERR__RESOLVE | Failed to resolve broker |
RD_KAFKA_RESP_ERR__MSG_TIMED_OUT | Produced message timed out |
RD_KAFKA_RESP_ERR__PARTITION_EOF | Reached the end of the topic+partition queue on the broker. Not really an error. This event is disabled by default, see the |
RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION | Permanent: Partition does not exist in cluster. |
RD_KAFKA_RESP_ERR__FS | File or filesystem error |
RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC | Permanent: Topic does not exist in cluster. |
RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN | All broker connections are down. |
RD_KAFKA_RESP_ERR__INVALID_ARG | Invalid argument, or invalid configuration |
RD_KAFKA_RESP_ERR__TIMED_OUT | Operation timed out |
RD_KAFKA_RESP_ERR__QUEUE_FULL | Queue is full |
RD_KAFKA_RESP_ERR__ISR_INSUFF | ISR count < required.acks |
RD_KAFKA_RESP_ERR__NODE_UPDATE | Broker node update |
RD_KAFKA_RESP_ERR__SSL | SSL error |
RD_KAFKA_RESP_ERR__WAIT_COORD | Waiting for coordinator to become available. |
RD_KAFKA_RESP_ERR__UNKNOWN_GROUP | Unknown client group |
RD_KAFKA_RESP_ERR__IN_PROGRESS | Operation in progress |
RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS | Previous operation in progress, wait for it to finish. |
RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION | This operation would interfere with an existing subscription |
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS | Assigned partitions (rebalance_cb) |
RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS | Revoked partitions (rebalance_cb) |
RD_KAFKA_RESP_ERR__CONFLICT | Conflicting use |
RD_KAFKA_RESP_ERR__STATE | Wrong state |
RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL | Unknown protocol |
RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED | Not implemented |
RD_KAFKA_RESP_ERR__AUTHENTICATION | Authentication failure |
RD_KAFKA_RESP_ERR__NO_OFFSET | No stored offset |
RD_KAFKA_RESP_ERR__OUTDATED | Outdated |
RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE | Timed out in queue |
RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE | Feature not supported by broker |
RD_KAFKA_RESP_ERR__WAIT_CACHE | Awaiting cache update |
RD_KAFKA_RESP_ERR__INTR | Operation interrupted (e.g., due to yield)) |
RD_KAFKA_RESP_ERR__KEY_SERIALIZATION | Key serialization error |
RD_KAFKA_RESP_ERR__VALUE_SERIALIZATION | Value serialization error |
RD_KAFKA_RESP_ERR__KEY_DESERIALIZATION | Key deserialization error |
RD_KAFKA_RESP_ERR__VALUE_DESERIALIZATION | Value deserialization error |
RD_KAFKA_RESP_ERR__PARTIAL | Partial response |
RD_KAFKA_RESP_ERR__READ_ONLY | Modification attempted on read-only object |
RD_KAFKA_RESP_ERR__NOENT | No such entry / item not found |
RD_KAFKA_RESP_ERR__UNDERFLOW | Read underflow |
RD_KAFKA_RESP_ERR__INVALID_TYPE | Invalid type |
RD_KAFKA_RESP_ERR__RETRY | Retry operation |
RD_KAFKA_RESP_ERR__PURGE_QUEUE | Purged in queue |
RD_KAFKA_RESP_ERR__PURGE_INFLIGHT | Purged in flight |
RD_KAFKA_RESP_ERR__FATAL | Fatal error: see rd_kafka_fatal_error() |
RD_KAFKA_RESP_ERR__INCONSISTENT | Inconsistent state |
RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE | Gap-less ordering would not be guaranteed if proceeding |
RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED | Maximum poll interval exceeded |
RD_KAFKA_RESP_ERR__UNKNOWN_BROKER | Unknown broker |
RD_KAFKA_RESP_ERR__NOT_CONFIGURED | Functionality not configured |
RD_KAFKA_RESP_ERR__FENCED | Instance has been fenced |
RD_KAFKA_RESP_ERR__APPLICATION | Application generated error |
RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST | Assignment lost |
RD_KAFKA_RESP_ERR__NOOP | No operation performed |
RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET | No offset to automatically reset to |
RD_KAFKA_RESP_ERR__LOG_TRUNCATION | Partition log truncation detected |
RD_KAFKA_RESP_ERR__INVALID_DIFFERENT_RECORD | A different record in the batch was invalid and this message failed persisting. |
RD_KAFKA_RESP_ERR__END | End internal error codes |
RD_KAFKA_RESP_ERR_UNKNOWN | Unknown broker error |
RD_KAFKA_RESP_ERR_NO_ERROR | Success |
RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE | Offset out of range |
RD_KAFKA_RESP_ERR_INVALID_MSG | Invalid message |
RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART | Unknown topic or partition |
RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE | Invalid message size |
RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE | Leader not available |
RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT | Request timed out |
RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE | Broker not available |
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE | Replica not available |
RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE | Message size too large |
RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH | StaleControllerEpochCode |
RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE | Offset metadata string too large |
RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION | Broker disconnected before response received |
RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS | Coordinator load in progress |
RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE | Coordinator not available |
RD_KAFKA_RESP_ERR_NOT_COORDINATOR | Not coordinator |
RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION | Invalid topic |
RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE | Message batch larger than configured server segment size |
RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS | Not enough in-sync replicas |
RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND | Message(s) written to insufficient number of in-sync replicas |
RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS | Invalid required acks value |
RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION | Specified group generation id is not valid |
RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL | Inconsistent group protocol |
RD_KAFKA_RESP_ERR_INVALID_GROUP_ID | Invalid group.id |
RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID | Unknown member |
RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT | Invalid session timeout |
RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS | Group rebalance in progress |
RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE | Commit offset data size is not valid |
RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED | Topic authorization failed |
RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED | Group authorization failed |
RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED | Cluster authorization failed |
RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP | Invalid timestamp |
RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM | Unsupported SASL mechanism |
RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE | Illegal SASL state |
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION | Unuspported version |
RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS | Topic already exists |
RD_KAFKA_RESP_ERR_INVALID_PARTITIONS | Invalid number of partitions |
RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR | Invalid replication factor |
RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT | Invalid replica assignment |
RD_KAFKA_RESP_ERR_INVALID_CONFIG | Invalid config |
RD_KAFKA_RESP_ERR_NOT_CONTROLLER | Not controller for cluster |
RD_KAFKA_RESP_ERR_INVALID_REQUEST | Invalid request |
RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT | Message format on broker does not support request |
RD_KAFKA_RESP_ERR_POLICY_VIOLATION | Policy violation |
RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER | Broker received an out of order sequence number |
RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER | Broker received a duplicate sequence number |
RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH | Producer attempted an operation with an old epoch |
RD_KAFKA_RESP_ERR_INVALID_TXN_STATE | Producer attempted a transactional operation in an invalid state |
RD_KAFKA_RESP_ERR_INVALID_PRODUCER_ID_MAPPING | Producer attempted to use a producer id which is not currently assigned to its transactional id |
RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT | Transaction timeout is larger than the maximum value allowed by the broker's max.transaction.timeout.ms |
RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS | Producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing |
RD_KAFKA_RESP_ERR_TRANSACTION_COORDINATOR_FENCED | Indicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer |
RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED | Transactional Id authorization failed |
RD_KAFKA_RESP_ERR_SECURITY_DISABLED | Security features are disabled |
RD_KAFKA_RESP_ERR_OPERATION_NOT_ATTEMPTED | Operation not attempted |
RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR | Disk error when trying to access log file on the disk |
RD_KAFKA_RESP_ERR_LOG_DIR_NOT_FOUND | The user-specified log directory is not found in the broker config |
RD_KAFKA_RESP_ERR_SASL_AUTHENTICATION_FAILED | SASL Authentication failed |
RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID | Unknown Producer Id |
RD_KAFKA_RESP_ERR_REASSIGNMENT_IN_PROGRESS | Partition reassignment is in progress |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTH_DISABLED | Delegation Token feature is not enabled |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_NOT_FOUND | Delegation Token is not found on server |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_OWNER_MISMATCH | Specified Principal is not valid Owner/Renewer |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_REQUEST_NOT_ALLOWED | Delegation Token requests are not allowed on this connection |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_AUTHORIZATION_FAILED | Delegation Token authorization failed |
RD_KAFKA_RESP_ERR_DELEGATION_TOKEN_EXPIRED | Delegation Token is expired |
RD_KAFKA_RESP_ERR_INVALID_PRINCIPAL_TYPE | Supplied principalType is not supported |
RD_KAFKA_RESP_ERR_NON_EMPTY_GROUP | The group is not empty |
RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND | The group id does not exist |
RD_KAFKA_RESP_ERR_FETCH_SESSION_ID_NOT_FOUND | The fetch session ID was not found |
RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH | The fetch session epoch is invalid |
RD_KAFKA_RESP_ERR_LISTENER_NOT_FOUND | No matching listener |
RD_KAFKA_RESP_ERR_TOPIC_DELETION_DISABLED | Topic deletion is disabled |
RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH | Leader epoch is older than broker epoch |
RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH | Leader epoch is newer than broker epoch |
RD_KAFKA_RESP_ERR_UNSUPPORTED_COMPRESSION_TYPE | Unsupported compression type |
RD_KAFKA_RESP_ERR_STALE_BROKER_EPOCH | Broker epoch has changed |
RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE | Leader high watermark is not caught up |
RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED | Group member needs a valid member ID |
RD_KAFKA_RESP_ERR_PREFERRED_LEADER_NOT_AVAILABLE | Preferred leader was not available |
RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED | Consumer group has reached maximum size |
RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID | Static consumer fenced by other consumer with same group.instance.id. |
RD_KAFKA_RESP_ERR_ELIGIBLE_LEADERS_NOT_AVAILABLE | Eligible partition leaders are not available |
RD_KAFKA_RESP_ERR_ELECTION_NOT_NEEDED | Leader election not needed for topic partition |
RD_KAFKA_RESP_ERR_NO_REASSIGNMENT_IN_PROGRESS | No partition reassignment is in progress |
RD_KAFKA_RESP_ERR_GROUP_SUBSCRIBED_TO_TOPIC | Deleting offsets of a topic while the consumer group is subscribed to it |
RD_KAFKA_RESP_ERR_INVALID_RECORD | Broker failed to validate record |
RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT | There are unstable offsets that need to be cleared |
RD_KAFKA_RESP_ERR_THROTTLING_QUOTA_EXCEEDED | Throttling quota has been exceeded |
RD_KAFKA_RESP_ERR_PRODUCER_FENCED | There is a newer producer with the same transactionalId which fences the current one |
RD_KAFKA_RESP_ERR_RESOURCE_NOT_FOUND | Request illegally referred to resource that does not exist |
RD_KAFKA_RESP_ERR_DUPLICATE_RESOURCE | Request illegally referred to the same resource twice |
RD_KAFKA_RESP_ERR_UNACCEPTABLE_CREDENTIAL | Requested credential would not meet criteria for acceptability |
RD_KAFKA_RESP_ERR_INCONSISTENT_VOTER_SET | Indicates that the either the sender or recipient of a voter-only request is not one of the expected voters |
RD_KAFKA_RESP_ERR_INVALID_UPDATE_VERSION | Invalid update version |
RD_KAFKA_RESP_ERR_FEATURE_UPDATE_FAILED | Unable to update finalized features due to server error |
RD_KAFKA_RESP_ERR_PRINCIPAL_DESERIALIZATION_FAILURE | Request principal deserialization failed during forwarding |
RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID | Unknown Topic Id |
RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH | The member epoch is fenced by the group coordinator |
RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID | The instance ID is still used by another member in the consumer group |
RD_KAFKA_RESP_ERR_UNSUPPORTED_ASSIGNOR | The assignor or its version range is not supported by the consumer group |
RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH | The member epoch is stale |
RD_KAFKA_RESP_ERR_UNKNOWN_SUBSCRIPTION_ID | Client sent a push telemetry request with an invalid or outdated subscription ID. |
RD_KAFKA_RESP_ERR_TELEMETRY_TOO_LARGE | Client sent a push telemetry request larger than the maximum size the broker will accept. |
enum rd_kafka_vtype_t |
Var-arg tag types.
Message persistence status can be used by the application to find out if a produced message was persisted in the topic log.
enum rd_kafka_conf_res_t |
enum rd_kafka_cert_type_t |
SSL certificate type.
Enumerator | |
---|---|
RD_KAFKA_CERT_PUBLIC_KEY | Client's public key |
RD_KAFKA_CERT_PRIVATE_KEY | Client's private key |
RD_KAFKA_CERT_CA | CA certificate |
enum rd_kafka_cert_enc_t |
SSL certificate encoding.
Enumerator | |
---|---|
RD_KAFKA_CERT_ENC_PKCS12 | PKCS#12 |
RD_KAFKA_CERT_ENC_DER | DER / binary X.509 ASN1 |
RD_KAFKA_CERT_ENC_PEM | PEM |
enum rd_kafka_admin_op_t |
Admin operation enum name for use with rd_kafka_AdminOptions_new()
IsolationLevel enum name for use with rd_kafka_AdminOptions_new()
Apache Kafka ACL operation types. Common type for multiple Admin API functions.
Apache Kafka config sources.
Apache Kafka resource types.
Apache Kafka pattern types.
RD_EXPORT int rd_kafka_version | ( | void | ) |
Returns the librdkafka version as integer.
const RD_EXPORT char* rd_kafka_version_str | ( | void | ) |
Returns the librdkafka version as string.
const RD_EXPORT char* rd_kafka_get_debug_contexts | ( | void | ) |
Retrieve supported debug contexts for use with the "debug"
configuration property. (runtime)
const RD_EXPORT char* rd_kafka_err2str | ( | rd_kafka_resp_err_t | err | ) |
Returns a human readable representation of a kafka error.
err | Error code to translate |
const RD_EXPORT char* rd_kafka_err2name | ( | rd_kafka_resp_err_t | err | ) |
Returns the error code name (enum name).
err | Error code to translate |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_last_error | ( | void | ) |
Returns the last error code generated by a legacy API call in the current thread.
The legacy APIs are the ones using errno to propagate error value, namely:
The main use for this function is to avoid converting system errno
values to rd_kafka_resp_err_t codes for legacy APIs.
RD_EXPORT RD_DEPRECATED rd_kafka_resp_err_t rd_kafka_errno2err | ( | int | errnox | ) |
Converts the system errno value errnox
to a rd_kafka_resp_err_t error code upon failure from the following functions:
errnox | System errno value to convert |
errnox
RD_EXPORT RD_DEPRECATED int rd_kafka_errno | ( | void | ) |
Returns the thread-local system errno.
On most platforms this is the same as errno
but in case of different runtimes between library and application (e.g., Windows static DLLs) this provides a means for exposing the errno librdkafka uses.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_fatal_error | ( | rd_kafka_t * | rk, |
char * | errstr, | ||
size_t | errstr_size | ||
) |
Returns the first fatal error set on this client instance, or RD_KAFKA_RESP_ERR_NO_ERROR if no fatal error has occurred.
This function is to be used with the Idempotent Producer and error_cb
to detect fatal errors.
Generally all errors raised by error_cb
are to be considered informational and temporary, the client will try to recover from all errors in a graceful fashion (by retrying, etc).
However, some errors should logically be considered fatal to retain consistency; in particular a set of errors that may occur when using the Idempotent Producer and the in-order or exactly-once producer guarantees can't be satisfied.
rk | Client instance. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written to if there is a fatal error. |
errstr_size | Writable size in errstr . |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_test_fatal_error | ( | rd_kafka_t * | rk, |
rd_kafka_resp_err_t | err, | ||
const char * | reason | ||
) |
Trigger a fatal error for testing purposes.
Since there is no practical way to trigger real fatal errors in the idempotent producer, this method allows an application to trigger fabricated fatal errors in tests to check its error handling code.
rk | Client instance. |
err | The underlying error code. |
reason | A human readable error reason. Will be prefixed with "test_fatal_error: " to differentiate from real fatal errors. |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_error_code | ( | const rd_kafka_error_t * | error | ) |
error
or RD_KAFKA_RESP_ERR_NO_ERROR if error
is NULL. const RD_EXPORT char* rd_kafka_error_name | ( | const rd_kafka_error_t * | error | ) |
error
, e.g, "ERR_UNKNOWN_MEMBER_ID", or an empty string if error
is NULL.const RD_EXPORT char* rd_kafka_error_string | ( | const rd_kafka_error_t * | error | ) |
error
, or an empty string if error
is NULL.RD_EXPORT int rd_kafka_error_is_fatal | ( | const rd_kafka_error_t * | error | ) |
error
is NULL). RD_EXPORT int rd_kafka_error_is_retriable | ( | const rd_kafka_error_t * | error | ) |
error
is NULL). RD_EXPORT int rd_kafka_error_txn_requires_abort | ( | const rd_kafka_error_t * | error | ) |
error
is NULL).RD_EXPORT void rd_kafka_error_destroy | ( | rd_kafka_error_t * | error | ) |
Free and destroy an error object.
error
. RD_EXPORT rd_kafka_error_t* rd_kafka_error_new | ( | rd_kafka_resp_err_t | code, |
const char * | fmt, | ||
... | |||
) |
Create a new error object with error code
and optional human readable error string in fmt
.
This method is mainly to be used for mocking errors in application test code.
The returned object must be destroyed with rd_kafka_error_destroy().
RD_EXPORT void rd_kafka_topic_partition_destroy | ( | rd_kafka_topic_partition_t * | rktpar | ) |
Destroy a rd_kafka_topic_partition_t.
RD_EXPORT void rd_kafka_topic_partition_set_leader_epoch | ( | rd_kafka_topic_partition_t * | rktpar, |
int32_t | leader_epoch | ||
) |
Sets the offset leader epoch (use -1 to clear).
rktpar | Partition object. |
leader_epoch | Offset leader epoch, use -1 to reset. |
RD_EXPORT int32_t rd_kafka_topic_partition_get_leader_epoch | ( | const rd_kafka_topic_partition_t * | rktpar | ) |
rktpar | Partition object. |
RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_topic_partition_list_new | ( | int | size | ) |
Create a new list/vector Topic+Partition container.
size | Initial allocated size used when the expected number of elements is known or can be estimated. Avoids reallocation and possibly relocation of the elems array. |
RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_topic_partition_list_add | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
const char * | topic, | ||
int32_t | partition | ||
) |
Add topic+partition to list.
rktparlist | List to extend |
topic | Topic name (copied) |
partition | Partition id |
RD_EXPORT void rd_kafka_topic_partition_list_add_range | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
const char * | topic, | ||
int32_t | start, | ||
int32_t | stop | ||
) |
Add range of partitions from start
to stop
inclusive.
rktparlist | List to extend |
topic | Topic name (copied) |
start | Start partition of range |
stop | Last partition of range (inclusive) |
RD_EXPORT int rd_kafka_topic_partition_list_del | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
const char * | topic, | ||
int32_t | partition | ||
) |
Delete partition from list.
rktparlist | List to modify |
topic | Topic name to match |
partition | Partition to match |
RD_EXPORT int rd_kafka_topic_partition_list_del_by_idx | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
int | idx | ||
) |
Delete partition from list by elems[] index.
RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_topic_partition_list_copy | ( | const rd_kafka_topic_partition_list_t * | src | ) |
Make a copy of an existing list.
src | The existing list to copy. |
src
RD_EXPORT rd_kafka_resp_err_t rd_kafka_topic_partition_list_set_offset | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
const char * | topic, | ||
int32_t | partition, | ||
int64_t | offset | ||
) |
Set offset to offset
for topic
and partition
.
partition
was not found in the list. RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_topic_partition_list_find | ( | const rd_kafka_topic_partition_list_t * | rktparlist, |
const char * | topic, | ||
int32_t | partition | ||
) |
Find element by topic
and partition
.
RD_EXPORT void rd_kafka_topic_partition_list_sort | ( | rd_kafka_topic_partition_list_t * | rktparlist, |
int(*)(const void *a, const void *b, void *cmp_opaque) | cmp, | ||
void * | cmp_opaque | ||
) |
Sort list using comparator cmp
.
If cmp
is NULL the default comparator will be used that sorts by ascending topic name and partition.
cmp_opaque
is provided as the cmp_opaque
argument to cmp
.
RD_EXPORT rd_kafka_headers_t* rd_kafka_headers_new | ( | size_t | initial_count | ) |
Create a new headers list.
initial_count | Preallocate space for this number of headers. Any number of headers may be added, updated and removed regardless of the initial count. |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_add | ( | rd_kafka_headers_t * | hdrs, |
const char * | name, | ||
ssize_t | name_size, | ||
const void * | value, | ||
ssize_t | value_size | ||
) |
Add header with name name
and value val
(copied) of size size
(not including null-terminator).
hdrs | Headers list. |
name | Header name. |
name_size | Header name size (not including the null-terminator). If -1 the name length is automatically acquired using strlen(). |
value | Pointer to header value, or NULL (set size to 0 or -1). |
value_size | Size of header value. If -1 the value is assumed to be a null-terminated string and the length is automatically acquired using strlen(). |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_remove | ( | rd_kafka_headers_t * | hdrs, |
const char * | name | ||
) |
Remove all headers for the given key (if any).
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get_last | ( | const rd_kafka_headers_t * | hdrs, |
const char * | name, | ||
const void ** | valuep, | ||
size_t * | sizep | ||
) |
Find last header in list hdrs
matching name
.
hdrs | Headers list. |
name | Header to find (last match). |
valuep | (out) Set to a (null-terminated) const pointer to the value (may be NULL). |
sizep | (out) Set to the value's size (not including null-terminator). |
valuep
includes a trailing null-terminator that is not accounted for in sizep
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get | ( | const rd_kafka_headers_t * | hdrs, |
size_t | idx, | ||
const char * | name, | ||
const void ** | valuep, | ||
size_t * | sizep | ||
) |
Iterator for headers matching name
.
Same semantics as rd_kafka_header_get_last()
hdrs | Headers to iterate. |
idx | Iterator index, start at 0 and increment by one for each call as long as RD_KAFKA_RESP_ERR_NO_ERROR is returned. |
name | Header name to match. |
valuep | (out) Set to a (null-terminated) const pointer to the value (may be NULL). |
sizep | (out) Set to the value's size (not including null-terminator). |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_header_get_all | ( | const rd_kafka_headers_t * | hdrs, |
size_t | idx, | ||
const char ** | namep, | ||
const void ** | valuep, | ||
size_t * | sizep | ||
) |
const RD_EXPORT char* rd_kafka_message_errstr | ( | const rd_kafka_message_t * | rkmessage | ) |
Returns the error string for an errored rd_kafka_message_t or NULL if there was no error.
const RD_EXPORT char* rd_kafka_message_produce_errstr | ( | const rd_kafka_message_t * | rkmessage | ) |
Returns the error string for an errored produced rd_kafka_message_t or NULL if there was no error.
RD_EXPORT int64_t rd_kafka_message_timestamp | ( | const rd_kafka_message_t * | rkmessage, |
rd_kafka_timestamp_type_t * | tstype | ||
) |
Returns the message timestamp for a consumed message.
The timestamp is the number of milliseconds since the epoch (UTC).
tstype
(if not NULL) is updated to indicate the type of timestamp.
RD_EXPORT int64_t rd_kafka_message_latency | ( | const rd_kafka_message_t * | rkmessage | ) |
Returns the latency for a produced message measured from the produce() call.
RD_EXPORT int32_t rd_kafka_message_broker_id | ( | const rd_kafka_message_t * | rkmessage | ) |
Returns the broker id of the broker the message was produced to or fetched from.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_message_headers | ( | const rd_kafka_message_t * | rkmessage, |
rd_kafka_headers_t ** | hdrsp | ||
) |
Get the message header list.
The returned pointer in *hdrsp
is associated with the rkmessage
and must not be used after destruction of the message object or the header list is replaced with rd_kafka_message_set_headers().
RD_EXPORT rd_kafka_resp_err_t rd_kafka_message_detach_headers | ( | rd_kafka_message_t * | rkmessage, |
rd_kafka_headers_t ** | hdrsp | ||
) |
Get the message header list and detach the list from the message making the application the owner of the headers. The application must eventually destroy the headers using rd_kafka_headers_destroy(). The message's headers will be set to NULL.
Otherwise same semantics as rd_kafka_message_headers()
RD_EXPORT void rd_kafka_message_set_headers | ( | rd_kafka_message_t * | rkmessage, |
rd_kafka_headers_t * | hdrs | ||
) |
Replace the message's current headers with a new list.
rkmessage | The message to set headers. |
hdrs | New header list. The message object assumes ownership of the list, the list will be destroyed automatically with the message object. The new headers list may be updated until the message object is passed or returned to librdkafka. |
RD_EXPORT size_t rd_kafka_header_cnt | ( | const rd_kafka_headers_t * | hdrs | ) |
Returns the number of header key/value pairs.
hdrs | Headers to count |
RD_EXPORT rd_kafka_msg_status_t rd_kafka_message_status | ( | const rd_kafka_message_t * | rkmessage | ) |
Returns the message's persistence status in the topic log.
RD_EXPORT int32_t rd_kafka_message_leader_epoch | ( | const rd_kafka_message_t * | rkmessage | ) |
const RD_EXPORT char* rd_kafka_Uuid_base64str | ( | const rd_kafka_Uuid_t * | uuid | ) |
Computes base64 encoding for the given uuid string.
uuid | UUID for which base64 encoding is required. |
RD_EXPORT int64_t rd_kafka_Uuid_least_significant_bits | ( | const rd_kafka_Uuid_t * | uuid | ) |
Gets least significant 64 bits for the given UUID.
uuid | UUID |
RD_EXPORT int64_t rd_kafka_Uuid_most_significant_bits | ( | const rd_kafka_Uuid_t * | uuid | ) |
Gets most significant 64 bits for the given UUID.
uuid | UUID |
RD_EXPORT rd_kafka_Uuid_t* rd_kafka_Uuid_new | ( | int64_t | most_significant_bits, |
int64_t | least_significant_bits | ||
) |
Creates a new UUID.
most_significant_bits | most significant 64 bits of the 128 bits UUID. |
least_significant_bits | least significant 64 bits of the 128 bits UUID. |
RD_EXPORT rd_kafka_Uuid_t* rd_kafka_Uuid_copy | ( | const rd_kafka_Uuid_t * | uuid | ) |
Copies the given UUID.
uuid | UUID to be copied. |
RD_EXPORT void rd_kafka_Uuid_destroy | ( | rd_kafka_Uuid_t * | uuid | ) |
Destroy the provided uuid.
uuid | UUID |
RD_EXPORT rd_kafka_conf_t* rd_kafka_conf_new | ( | void | ) |
Create configuration object.
When providing your own configuration to the rd_kafka_*_new_*
() calls the rd_kafka_conf_t objects needs to be created with this function which will set up the defaults. I.e.:
Please see CONFIGURATION.md for the default settings or use rd_kafka_conf_properties_show() to provide the information at runtime.
The properties are identical to the Apache Kafka configuration properties whenever possible.
RD_EXPORT rd_kafka_conf_t* rd_kafka_conf_dup | ( | const rd_kafka_conf_t * | conf | ) |
Creates a copy/duplicate of configuration object conf
.
const RD_EXPORT rd_kafka_conf_t* rd_kafka_conf | ( | rd_kafka_t * | rk | ) |
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set | ( | rd_kafka_conf_t * | conf, |
const char * | name, | ||
const char * | value, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Sets a configuration property.
conf
must have been previously created with rd_kafka_conf_new().
Fallthrough: Topic-level configuration properties may be set using this interface in which case they are applied on the default_topic_conf
. If no default_topic_conf
has been set one will be created. Any subsequent rd_kafka_conf_set_default_topic_conf() calls will replace the current default topic configuration.
rd_kafka_conf_res_t
to indicate success or failure. In case of failure errstr
is updated to contain a human readable error string.RD_EXPORT void rd_kafka_conf_set_background_event_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, rd_kafka_event_t *rkev, void *opaque) | event_cb | ||
) |
Generic event callback to be used with the event API to trigger callbacks for rd_kafka_event_t
objects from a background thread serving the background queue.
How to use:
The application must destroy the rkev
passed to event
cb using rd_kafka_event_destroy().
The event_cb
opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
event_cb
will be called spontaneously from a background thread completely managed by librdkafka. Take care to perform proper locking of application objects.RD_EXPORT void rd_kafka_conf_set_dr_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) | dr_cb | ||
) |
RD_EXPORT void rd_kafka_conf_set_dr_msg_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, const rd_kafka_message_t *rkmessage, void *opaque) | dr_msg_cb | ||
) |
Producer: Set delivery report callback in provided conf
object.
The delivery report callback will be called once for each message accepted by rd_kafka_produce() (et.al) with err
set to indicate the result of the produce request.
The callback is called when a message is succesfully produced or if librdkafka encountered a permanent failure. Delivery errors occur when the retry count is exceeded, when the message.timeout.ms timeout is exceeded or there is a permanent error like RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART.
An application must call rd_kafka_poll() at regular intervals to serve queued delivery report callbacks.
The broker-assigned offset can be retrieved with rkmessage->offset
and the timestamp can be retrieved using rd_kafka_message_timestamp().
The dr_msg_cb
opaque
argument is the opaque set with rd_kafka_conf_set_opaque(). The per-message msg_opaque value is available in rd_kafka_message_t._private
.
RD_EXPORT void rd_kafka_conf_set_consume_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_message_t *rkmessage, void *opaque) | consume_cb | ||
) |
Consumer: Set consume callback for use with rd_kafka_consumer_poll()
The consume_cb
opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_rebalance_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *partitions, void *opaque) | rebalance_cb | ||
) |
Consumer: Set rebalance callback for use with coordinated consumer group balancing.
The err
field is set to either RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS or RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS and 'partitions' contains the full partition set that was either assigned or revoked.
Registering a rebalance_cb
turns off librdkafka's automatic partition assignment/revocation and instead delegates that responsibility to the application's rebalance_cb
.
The rebalance callback is responsible for updating librdkafka's assignment set based on the two events: RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS and RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS but should also be able to handle arbitrary rebalancing failures where err
is neither of those.
For eager/non-cooperative partition.assignment.strategy
assignors, such as range
and roundrobin
, the application must use rd_kafka_assign() to set or clear the entire assignment. For the cooperative assignors, such as cooperative-sticky
, the application must use rd_kafka_incremental_assign() for RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS and rd_kafka_incremental_unassign() for RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS.
Without a rebalance callback this is done automatically by librdkafka but registering a rebalance callback gives the application flexibility in performing other operations along with the assigning/revocation, such as fetching offsets from an alternate location (on assign) or manually committing offsets (on revoke).
rebalance_cb is always triggered exactly once when a rebalance completes with a new assignment, even if that assignment is empty. If an eager/non-cooperative assignor is configured, there will eventually be exactly one corresponding call to rebalance_cb to revoke these partitions (even if empty), whether this is due to a group rebalance or lost partitions. In the cooperative case, rebalance_cb will never be called if the set of partitions being revoked is empty (whether or not lost).
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
partitions
list is destroyed by librdkafka on return return from the rebalance_cb and must not be freed or saved by the application.partitions
list. Changing this list should only be done to change the initial offsets for each partition. But a function like rd_kafka_position()
might have unexpected effects for instance when a consumer gets assigned a partition it used to consume at an earlier rebalance. In this case, the list of partitions will be updated with the old offset for that partition. In this case, it is generally better to pass a copy of the list (see rd_kafka_topic_partition_list_copy()
). The result of rd_kafka_position()
is typically outdated in RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS.The following example shows the application's responsibilities:
RD_EXPORT void rd_kafka_conf_set_offset_commit_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *opaque) | offset_commit_cb | ||
) |
Consumer: Set offset commit callback for use with consumer groups.
The results of automatic or manual offset commits will be scheduled for this callback and is served by rd_kafka_consumer_poll().
If no partitions had valid offsets to commit this callback will be called with err
== RD_KAFKA_RESP_ERR__NO_OFFSET which is not to be considered an error.
The offsets
list contains per-partition information:
offset:
committed offset (attempted)err:
commit errorThe callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_error_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, int err, const char *reason, void *opaque) | error_cb | ||
) |
Set error callback in provided conf object.
The error callback is used by librdkafka to signal warnings and errors back to the application.
These errors should generally be considered informational and non-permanent, the client will try to recover automatically from all type of errors. Given that the client and cluster configuration is correct the application should treat these as temporary errors.
error_cb
will be triggered with err
set to RD_KAFKA_RESP_ERR__FATAL if a fatal error has been raised; in this case use rd_kafka_fatal_error() to retrieve the fatal error code and error string, and then begin terminating the client instance.
If no error_cb
is registered, or RD_KAFKA_EVENT_ERROR has not been set with rd_kafka_conf_set_events, then the errors will be logged instead.
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_throttle_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int throttle_time_ms, void *opaque) | throttle_cb | ||
) |
Set throttle callback.
The throttle callback is used to forward broker throttle times to the application for Produce and Fetch (consume) requests.
Callbacks are triggered whenever a non-zero throttle time is returned by the broker, or when the throttle time drops back to zero.
An application must call rd_kafka_poll() or rd_kafka_consumer_poll() at regular intervals to serve queued callbacks.
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_log_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(const rd_kafka_t *rk, int level, const char *fac, const char *buf) | log_cb | ||
) |
Set logger callback.
The default is to print to stderr, but a syslog logger is also available, see rd_kafka_log_print and rd_kafka_log_syslog for the builtin alternatives. Alternatively the application may provide its own logger callback. Or pass func
as NULL to disable logging.
This is the configuration alternative to the deprecated rd_kafka_set_logger()
rd_kafka_set_log_queue()
. An application MUST NOT call any librdkafka APIs or do any prolonged work in a non-forwarded log_cb
. RD_EXPORT void rd_kafka_conf_set_stats_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) | stats_cb | ||
) |
Set statistics callback in provided conf object.
The statistics callback is triggered from rd_kafka_poll() every statistics.interval.ms
(needs to be configured separately). Function arguments:
rk
- Kafka handlejson
- String containing the statistics data in JSON formatjson_len
- Length of json
string.opaque
- Application-provided opaque as set by rd_kafka_conf_set_opaque().For more information on the format of json
, see https://github.com/confluentinc/librdkafka/wiki/Statistics
If the application wishes to hold on to the json
pointer and free it at a later time it must return 1 from the stats_cb
. If the application returns 0 from the stats_cb
then librdkafka will immediately free the json
pointer.
See STATISTICS.md for a full definition of the JSON object.
RD_EXPORT void rd_kafka_conf_set_oauthbearer_token_refresh_cb | ( | rd_kafka_conf_t * | conf, |
void(*)(rd_kafka_t *rk, const char *oauthbearer_config, void *opaque) | oauthbearer_token_refresh_cb | ||
) |
Set SASL/OAUTHBEARER token refresh callback in provided conf object.
conf | the configuration to mutate. |
oauthbearer_token_refresh_cb | the callback to set; callback function arguments:rk - Kafka handleoauthbearer_config - Value of configuration property sasl.oauthbearer.config. opaque - Application-provided opaque set via rd_kafka_conf_set_opaque() |
The SASL/OAUTHBEARER token refresh callback is triggered via rd_kafka_poll() whenever OAUTHBEARER is the SASL mechanism and a token needs to be retrieved, typically based on the configuration defined in sasl.oauthbearer.config
.
The callback should invoke rd_kafka_oauthbearer_set_token() or rd_kafka_oauthbearer_set_token_failure() to indicate success or failure, respectively.
The refresh operation is eventable and may be received via rd_kafka_queue_poll() with an event type of RD_KAFKA_EVENT_OAUTHBEARER_TOKEN_REFRESH
.
Note that before any SASL/OAUTHBEARER broker connection can succeed the application must call rd_kafka_oauthbearer_set_token() once – either directly or, more typically, by invoking either rd_kafka_poll(), rd_kafka_consumer_poll(), rd_kafka_queue_poll(), etc, in order to cause retrieval of an initial token to occur.
Alternatively, the application can enable the SASL queue by calling rd_kafka_conf_enable_sasl_queue() on the configuration object prior to creating the client instance, get the SASL queue with rd_kafka_queue_get_sasl(), and either serve the queue manually by calling rd_kafka_queue_poll(), or redirecting the queue to the background thread to have the queue served automatically. For the latter case the SASL queue must be forwarded to the background queue with rd_kafka_queue_forward(). A convenience function is available to automatically forward the SASL queue to librdkafka's background thread, see rd_kafka_sasl_background_callbacks_enable().
An unsecured JWT refresh handler is provided by librdkafka for development and testing purposes, it is enabled by setting the enable.sasl.oauthbearer.unsecure.jwt
property to true and is mutually exclusive to using a refresh callback.
RD_EXPORT void rd_kafka_conf_enable_sasl_queue | ( | rd_kafka_conf_t * | conf, |
int | enable | ||
) |
Enable/disable creation of a queue specific to SASL events and callbacks.
For SASL mechanisms that trigger callbacks (currently OAUTHBEARER) this configuration API allows an application to get a dedicated queue for the SASL events/callbacks. After enabling the queue with this API the application can retrieve the queue by calling rd_kafka_queue_get_sasl() on the client instance. This queue may then be served directly by the application (with rd_kafka_queue_poll(), et.al) or forwarded to another queue, such as the background queue.
A convenience function is available to automatically forward the SASL queue to librdkafka's background thread, see rd_kafka_sasl_background_callbacks_enable().
By default (enable
= 0) the main queue (as served by rd_kafka_poll(), et.al.) is used for SASL callbacks.
RD_EXPORT void rd_kafka_conf_set_socket_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(int domain, int type, int protocol, void *opaque) | socket_cb | ||
) |
Set socket callback.
The socket callback is responsible for opening a socket according to the supplied domain
, type
and protocol
. The socket shall be created with CLOEXEC
set in a racefree fashion, if possible.
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
Default:
RD_EXPORT void rd_kafka_conf_set_connect_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(int sockfd, const struct sockaddr *addr, int addrlen, const char *id, void *opaque) | connect_cb | ||
) |
Set connect callback.
The connect callback is responsible for connecting socket sockfd
to peer address addr
. The id
field contains the broker identifier.
connect_cb
shall return 0 on success (socket connected) or an error number (errno) on error.
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_closesocket_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(int sockfd, void *opaque) | closesocket_cb | ||
) |
Set close socket callback.
Close a socket (optionally opened with socket_cb()).
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_open_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(const char *pathname, int flags, mode_t mode, void *opaque) | open_cb | ||
) |
Set open callback.
The open callback is responsible for opening the file specified by pathname, flags and mode. The file shall be opened with CLOEXEC
set in a racefree fashion, if possible.
Default:
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
RD_EXPORT void rd_kafka_conf_set_resolve_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(const char *node, const char *service, const struct addrinfo *hints, struct addrinfo **res, void *opaque) | resolve_cb | ||
) |
Set address resolution callback.
The callback is responsible for resolving the hostname node
and the service service
into a list of socket addresses as getaddrinfo(3)
would. The hints
and res
parameters function as they do for getaddrinfo(3)
. The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
If the callback is invoked with a NULL node
, service
, and hints
, the callback should instead free the addrinfo struct specified in res
. In this case the callback must succeed; the return value will not be checked by the caller.
The callback's return value is interpreted as the return value of getaddrinfo(3)
.
RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set_ssl_cert_verify_cb | ( | rd_kafka_conf_t * | conf, |
int(*)(rd_kafka_t *rk, const char *broker_name, int32_t broker_id, int *x509_error, int depth, const char *buf, size_t size, char *errstr, size_t errstr_size, void *opaque) | ssl_cert_verify_cb | ||
) |
Sets the verification callback of the broker certificate.
The verification callback is triggered from internal librdkafka threads upon connecting to a broker. On each connection attempt the callback will be called for each certificate in the broker's certificate chain, starting at the root certification, as long as the application callback returns 1 (valid certificate). broker_name
and broker_id
correspond to the broker the connection is being made to. The x509_error
argument indicates if OpenSSL's verification of the certificate succeed (0) or failed (an OpenSSL error code). The application may set the SSL context error code by returning 0 from the verify callback and providing a non-zero SSL context error code in x509_error
. If the verify callback sets x509_error
to 0, returns 1, and the original x509_error
was non-zero, the error on the SSL context will be cleared. x509_error
is always a valid pointer to an int.
depth
is the depth of the current certificate in the chain, starting at the root certificate.
The certificate itself is passed in binary DER format in buf
of size size
.
The callback must return 1 if verification succeeds, or 0 if verification fails and then write a human-readable error message to errstr
(limited to errstr_size
bytes, including nul-term).
The callback's opaque
argument is the opaque set with rd_kafka_conf_set_opaque().
x509_error
codes. RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_set_ssl_cert | ( | rd_kafka_conf_t * | conf, |
rd_kafka_cert_type_t | cert_type, | ||
rd_kafka_cert_enc_t | cert_enc, | ||
const void * | buffer, | ||
size_t | size, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Set certificate/key cert_type
from the cert_enc
encoded memory at buffer
of size
bytes.
conf | Configuration object. |
cert_type | Certificate or key type to configure. |
cert_enc | Buffer encoding type. |
buffer | Memory pointer to encoded certificate or key. The memory is not referenced after this function returns. |
size | Size of memory at buffer . |
errstr | Memory were a human-readable error string will be written on failure. |
errstr_size | Size of errstr , including space for nul-terminator. |
buffer
is of incorrect encoding, or if librdkafka was not built with SSL support.cert_type
will replace the previous value.buffer
set to NULL will clear the configuration for cert_type
.ssl.key.password
configuration property prior to calling this function.ssl.key.pem
and ssl.certificate.pem
configuration properties.ssl.ca.pem
configuration property.RD_EXPORT void rd_kafka_conf_set_engine_callback_data | ( | rd_kafka_conf_t * | conf, |
void * | callback_data | ||
) |
Set callback_data for OpenSSL engine.
conf | Configuration object. |
callback_data | passed to engine callbacks, e.g. ENGINE_load_ssl_client_cert . |
ssl.engine.location
configuration must be set for this to have affect.value
must remain valid for the lifetime of the configuration object and any Kafka clients that use it. RD_EXPORT void rd_kafka_conf_set_opaque | ( | rd_kafka_conf_t * | conf, |
void * | opaque | ||
) |
Sets the application's opaque pointer that will be passed to callbacks.
RD_EXPORT void rd_kafka_conf_set_default_topic_conf | ( | rd_kafka_conf_t * | conf, |
rd_kafka_topic_conf_t * | tconf | ||
) |
Sets the default topic configuration to use for automatically subscribed topics (e.g., through pattern-matched topics). The topic config object is not usable after this call.
RD_EXPORT rd_kafka_topic_conf_t* rd_kafka_conf_get_default_topic_conf | ( | rd_kafka_conf_t * | conf | ) |
Gets the default topic configuration as previously set with rd_kafka_conf_set_default_topic_conf() or that was implicitly created by configuring a topic-level property on the global conf
object.
conf's
default topic configuration (if any), or NULL.conf
object. It may be modified but not destroyed and its lifetime is the same as the conf
object or the next call to rd_kafka_conf_set_default_topic_conf(). RD_EXPORT rd_kafka_conf_res_t rd_kafka_conf_get | ( | const rd_kafka_conf_t * | conf, |
const char * | name, | ||
char * | dest, | ||
size_t * | dest_size | ||
) |
Retrieve configuration value for property name
.
If dest
is non-NULL the value will be written to dest
with at most dest_size
.
*dest_size
is updated to the full length of the value, thus if *dest_size
initially is smaller than the full length the application may reallocate dest
to fit the returned *dest_size
and try again.
If dest
is NULL only the full length of the value is returned.
Fallthrough: Topic-level configuration properties from the default_topic_conf
may be retrieved using this interface.
RD_KAFKA_CONF_OK
if the property name matched, else RD_KAFKA_CONF_UNKNOWN
. RD_EXPORT rd_kafka_conf_res_t rd_kafka_topic_conf_get | ( | const rd_kafka_topic_conf_t * | conf, |
const char * | name, | ||
char * | dest, | ||
size_t * | dest_size | ||
) |
Retrieve topic configuration value for property name
.
const RD_EXPORT char** rd_kafka_conf_dump | ( | rd_kafka_conf_t * | conf, |
size_t * | cntp | ||
) |
Dump the configuration properties and values of conf
to an array with "key", "value" pairs.
The number of entries in the array is returned in *cntp
.
The dump must be freed with rd_kafka_conf_dump_free()
.
const RD_EXPORT char** rd_kafka_topic_conf_dump | ( | rd_kafka_topic_conf_t * | conf, |
size_t * | cntp | ||
) |
Dump the topic configuration properties and values of conf
to an array with "key", "value" pairs.
The number of entries in the array is returned in *cntp
.
The dump must be freed with rd_kafka_conf_dump_free()
.
RD_EXPORT void rd_kafka_conf_properties_show | ( | FILE * | fp | ) |
Prints a table to fp
of all supported configuration properties, their default values as well as a description.
RD_EXPORT rd_kafka_topic_conf_t* rd_kafka_topic_conf_new | ( | void | ) |
Create topic configuration object.
RD_EXPORT rd_kafka_conf_res_t rd_kafka_topic_conf_set | ( | rd_kafka_topic_conf_t * | conf, |
const char * | name, | ||
const char * | value, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Sets a single rd_kafka_topic_conf_t value by property name.
topic_conf
should have been previously set up with rd_kafka_topic_conf_new()
.
RD_EXPORT void rd_kafka_topic_conf_set_opaque | ( | rd_kafka_topic_conf_t * | conf, |
void * | rkt_opaque | ||
) |
Sets the application's opaque pointer that will be passed to all topic callbacks as the rkt_opaque
argument.
RD_EXPORT void rd_kafka_topic_conf_set_partitioner_cb | ( | rd_kafka_topic_conf_t * | topic_conf, |
int32_t(*)(const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) | partitioner | ||
) |
Producer: Set partitioner callback in provided topic conf object.
The partitioner may be called in any thread at any time, it may be called multiple times for the same message/key.
The callback's rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The callback's msg_opaque
argument is the per-message opaque passed to produce().
Partitioner function constraints:
RD_KAFKA_PARTITION_UA
value if partitioning could not be performed. RD_EXPORT void rd_kafka_topic_conf_set_msg_order_cmp | ( | rd_kafka_topic_conf_t * | topic_conf, |
int(*)(const rd_kafka_message_t *a, const rd_kafka_message_t *b) | msg_order_cmp | ||
) |
Producer: Set message queueing order comparator callback.
The callback may be called in any thread at any time, it may be called multiple times for the same message.
Ordering comparator function constraints:
The comparator shall compare the two messages and return:
a
should be inserted before message b
.a
should be inserted after message b
.queuing.strategy=fifo
new messages are enqueued to the tail of the queue regardless of msg_order_cmp, but retried messages are still affected by msg_order_cmp.RD_EXPORT int rd_kafka_topic_partition_available | ( | const rd_kafka_topic_t * | rkt, |
int32_t | partition | ||
) |
Check if partition is available (has a leader broker).
RD_EXPORT int32_t rd_kafka_msg_partitioner_random | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Random partitioner.
Will try not to return unavailable partitions.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1. RD_EXPORT int32_t rd_kafka_msg_partitioner_consistent | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Consistent partitioner.
Uses consistent hashing to map identical keys onto identical partitions.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1 based on the CRC value of the key RD_EXPORT int32_t rd_kafka_msg_partitioner_consistent_random | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Consistent-Random partitioner.
This is the default partitioner. Uses consistent hashing to map identical keys onto identical partitions, and messages without keys will be assigned via the random partitioner.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1 based on the CRC value of the key (if provided) RD_EXPORT int32_t rd_kafka_msg_partitioner_murmur2 | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Murmur2 partitioner (Java compatible).
Uses consistent hashing to map identical keys onto identical partitions using Java-compatible Murmur2 hashing.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1. RD_EXPORT int32_t rd_kafka_msg_partitioner_murmur2_random | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Consistent-Random Murmur2 partitioner (Java compatible).
Uses consistent hashing to map identical keys onto identical partitions using Java-compatible Murmur2 hashing. Messages without keys will be assigned via the random partitioner.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1. RD_EXPORT int32_t rd_kafka_msg_partitioner_fnv1a | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
FNV-1a partitioner.
Uses consistent hashing to map identical keys onto identical partitions using FNV-1a hashing.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1. RD_EXPORT int32_t rd_kafka_msg_partitioner_fnv1a_random | ( | const rd_kafka_topic_t * | rkt, |
const void * | key, | ||
size_t | keylen, | ||
int32_t | partition_cnt, | ||
void * | rkt_opaque, | ||
void * | msg_opaque | ||
) |
Consistent-Random FNV-1a partitioner.
Uses consistent hashing to map identical keys onto identical partitions using FNV-1a hashing. Messages without keys will be assigned via the random partitioner.
The rkt_opaque
argument is the opaque set by rd_kafka_topic_conf_set_opaque(). The msg_opaque
argument is the per-message opaque passed to produce().
partition_cnt
- 1. RD_EXPORT rd_kafka_t* rd_kafka_new | ( | rd_kafka_type_t | type, |
rd_kafka_conf_t * | conf, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Creates a new Kafka handle and starts its operation according to the specified type
(RD_KAFKA_CONSUMER
or RD_KAFKA_PRODUCER
).
conf
is an optional struct created with rd_kafka_conf_new()
that will be used instead of the default configuration. The conf
object is freed by this function on success and must not be used or destroyed by the application subsequently. See rd_kafka_conf_set()
et.al for more information.
errstr
must be a pointer to memory of at least size errstr_size
where rd_kafka_new()
may write a human readable error message in case the creation of a new handle fails. In which case the function returns NULL.
RD_KAFKA_CONSUMER
rd_kafka_t handle is created it may either operate in the legacy simple consumer mode using the rd_kafka_consume_start() interface, or the High-level KafkaConsumer API. errstr
)RD_EXPORT void rd_kafka_destroy | ( | rd_kafka_t * | rk | ) |
Destroy Kafka handle.
group.id
was configured, and the rd_kafka_consumer_close() was not explicitly called by the application. This in turn may trigger consumer callbacks, such as rebalance_cb. Use rd_kafka_destroy_flags() with RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE to avoid this behaviour.RD_EXPORT void rd_kafka_destroy_flags | ( | rd_kafka_t * | rk, |
int | flags | ||
) |
Destroy Kafka handle according to specified destroy flags.
RD_EXPORT char* rd_kafka_memberid | ( | const rd_kafka_t * | rk | ) |
Returns this client's broker-assigned group member id.
free()
or rd_kafka_mem_free() RD_EXPORT char* rd_kafka_clusterid | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Returns the ClusterId as reported in broker metadata.
rk | Client instance. |
timeout_ms | If there is no cached value from metadata retrieval then this specifies the maximum amount of time (in milliseconds) the call will block waiting for metadata to be retrieved. Use 0 for non-blocking calls. |
RD_EXPORT int32_t rd_kafka_controllerid | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Returns the current ControllerId as reported in broker metadata.
rk | Client instance. |
timeout_ms | If there is no cached value from metadata retrieval then this specifies the maximum amount of time (in milliseconds) the call will block waiting for metadata to be retrieved. Use 0 for non-blocking calls. |
RD_EXPORT rd_kafka_topic_t* rd_kafka_topic_new | ( | rd_kafka_t * | rk, |
const char * | topic, | ||
rd_kafka_topic_conf_t * | conf | ||
) |
Creates a new topic handle for topic named topic
.
conf
is an optional configuration for the topic created with rd_kafka_topic_conf_new()
that will be used instead of the default topic configuration. The conf
object is freed by this function and must not be used or destroyed by the application subsequently. See rd_kafka_topic_conf_set()
et.al for more information.
Topic handles are refcounted internally and calling rd_kafka_topic_new() again with the same topic name will return the previous topic handle without updating the original handle's configuration. Applications must eventually call rd_kafka_topic_destroy() for each succesfull call to rd_kafka_topic_new() to clear up resources.
errno
to an rd_kafka_resp_err_t error code.RD_EXPORT void rd_kafka_topic_destroy | ( | rd_kafka_topic_t * | rkt | ) |
Loose application's topic handle refcount as previously created with rd_kafka_topic_new()
.
RD_EXPORT int rd_kafka_poll | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Polls the provided kafka handle for events.
Events will cause application-provided callbacks to be called.
The timeout_ms
argument specifies the maximum amount of time (in milliseconds) that the call will block waiting for events. For non-blocking calls, provide 0 as timeout_ms
. To wait indefinitely for an event, provide -1.
Events:
RD_EXPORT void rd_kafka_yield | ( | rd_kafka_t * | rk | ) |
Cancels the current callback dispatcher (rd_kafka_poll(), rd_kafka_consume_callback(), etc).
A callback may use this to force an immediate return to the calling code (caller of e.g. rd_kafka_poll()) without processing any further events.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_pause_partitions | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | partitions | ||
) |
Pause producing or consumption for the provided list of partitions.
Success or error is returned per-partition err
in the partitions
list.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_resume_partitions | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | partitions | ||
) |
Resume producing consumption for the provided list of partitions.
Success or error is returned per-partition err
in the partitions
list.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_query_watermark_offsets | ( | rd_kafka_t * | rk, |
const char * | topic, | ||
int32_t | partition, | ||
int64_t * | low, | ||
int64_t * | high, | ||
int | timeout_ms | ||
) |
Query broker for low (oldest/beginning) and high (newest/end) offsets for partition.
Offsets are returned in *low
and *high
respectively.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_get_watermark_offsets | ( | rd_kafka_t * | rk, |
const char * | topic, | ||
int32_t | partition, | ||
int64_t * | low, | ||
int64_t * | high | ||
) |
Get last known low (oldest/beginning) and high (newest/end) offsets for partition.
The low offset is updated periodically (if statistics.interval.ms is set) while the high offset is updated on each fetched message set from the broker.
If there is no cached offset (either low or high, or both) then RD_KAFKA_OFFSET_INVALID will be returned for the respective offset.
Offsets are returned in *low
and *high
respectively.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_offsets_for_times | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | offsets, | ||
int | timeout_ms | ||
) |
Look up the offsets for the given partitions by timestamp.
The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition.
The timestamps to query are represented as offset
in offsets
on input, and offset
will contain the offset on output.
The function will block for at most timeout_ms
milliseconds.
rd_kafka_topic_partition_t.err
timeout_ms
, RD_KAFKA_RESP_ERR__INVALID_ARG if the offsets
list is empty, RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION if all partitions are unknown, RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE if unable to query leaders for the given partitions. RD_EXPORT void* rd_kafka_mem_calloc | ( | rd_kafka_t * | rk, |
size_t | num, | ||
size_t | size | ||
) |
Allocate and zero memory using the same allocator librdkafka uses.
This is typically an abstraction for the calloc(3) call and makes sure the application can use the same memory allocator as librdkafka for allocating pointers that are used by librdkafka.
rk
can be set to return memory allocated by a specific rk
instance otherwise pass NULL for rk
.
RD_EXPORT void* rd_kafka_mem_malloc | ( | rd_kafka_t * | rk, |
size_t | size | ||
) |
Allocate memory using the same allocator librdkafka uses.
This is typically an abstraction for the malloc(3) call and makes sure the application can use the same memory allocator as librdkafka for allocating pointers that are used by librdkafka.
rk
can be set to return memory allocated by a specific rk
instance otherwise pass NULL for rk
.
RD_EXPORT void rd_kafka_mem_free | ( | rd_kafka_t * | rk, |
void * | ptr | ||
) |
Free pointer returned by librdkafka.
This is typically an abstraction for the free(3) call and makes sure the application can use the same memory allocator as librdkafka for freeing pointers returned by librdkafka.
In standard setups it is usually not necessary to use this interface rather than the free(3) functione.
rk
must be set for memory returned by APIs that take an rk
argument, for other APIs pass NULL for rk
.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_new | ( | rd_kafka_t * | rk | ) |
Create a new message queue.
See rd_kafka_consume_start_queue(), rd_kafka_consume_queue(), et.al.
RD_EXPORT void rd_kafka_queue_destroy | ( | rd_kafka_queue_t * | rkqu | ) |
Destroy a queue, purging all of its enqueued messages.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_main | ( | rd_kafka_t * | rk | ) |
Use rd_kafka_queue_destroy() to loose the reference.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_sasl | ( | rd_kafka_t * | rk | ) |
Use rd_kafka_queue_destroy() to loose the reference.
RD_EXPORT rd_kafka_error_t* rd_kafka_sasl_background_callbacks_enable | ( | rd_kafka_t * | rk | ) |
Enable SASL OAUTHBEARER refresh callbacks on the librdkafka background thread.
This serves as an alternative for applications that do not call rd_kafka_poll() (et.al.) at regular intervals (or not at all), as a means of automatically trigger the refresh callbacks, which are needed to initiate connections to the brokers in the case a custom OAUTHBEARER refresh callback is configured.
RD_EXPORT rd_kafka_error_t* rd_kafka_sasl_set_credentials | ( | rd_kafka_t * | rk, |
const char * | username, | ||
const char * | password | ||
) |
Sets SASL credentials used for SASL PLAIN and SCRAM mechanisms by this Kafka client.
This function sets or resets the SASL username and password credentials used by this Kafka client. The new credentials will be used the next time this client needs to authenticate to a broker. This function will not disconnect existing connections that might have been made using the old credentials.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_consumer | ( | rd_kafka_t * | rk | ) |
Use rd_kafka_queue_destroy() to loose the reference.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_partition | ( | rd_kafka_t * | rk, |
const char * | topic, | ||
int32_t | partition | ||
) |
Use rd_kafka_queue_destroy() to loose the reference.
RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_background | ( | rd_kafka_t * | rk | ) |
The background thread queue provides the application with an automatically polled queue that triggers the event callback in a background thread, this background thread is completely managed by librdkafka.
The background thread queue is automatically created if a generic event handler callback is configured with rd_kafka_conf_set_background_event_cb() or if rd_kafka_queue_get_background() is called.
The background queue is polled and served by librdkafka and MUST NOT be polled, forwarded, or otherwise managed by the application, it may only be used as the destination queue passed to queue-enabled APIs, such as the Admin API.
Use rd_kafka_queue_destroy() to loose the reference.
RD_EXPORT void rd_kafka_queue_forward | ( | rd_kafka_queue_t * | src, |
rd_kafka_queue_t * | dst | ||
) |
Forward/re-route queue src
to dst
. If dst
is NULL
the forwarding is removed.
The internal refcounts for both queues are increased.
dst
is NULL or not, after calling this function, src
will not forward it's fetch queue to the consumer queue. RD_EXPORT rd_kafka_resp_err_t rd_kafka_set_log_queue | ( | rd_kafka_t * | rk, |
rd_kafka_queue_t * | rkqu | ||
) |
Forward librdkafka logs (and debug) to the specified queue for serving with one of the ..poll() calls.
This allows an application to serve log callbacks (log_cb
) in its thread of choice.
rk | Client instance. |
rkqu | Queue to forward logs to. If the value is NULL the logs are forwarded to the main queue. |
log.queue
MUST also be set to true.RD_EXPORT size_t rd_kafka_queue_length | ( | rd_kafka_queue_t * | rkqu | ) |
RD_EXPORT void rd_kafka_queue_io_event_enable | ( | rd_kafka_queue_t * | rkqu, |
int | fd, | ||
const void * | payload, | ||
size_t | size | ||
) |
Enable IO event triggering for queue.
To ease integration with IO based polling loops this API allows an application to create a separate file-descriptor that librdkafka will write payload
(of size size
) to whenever a new element is enqueued on a previously empty queue.
To remove event triggering call with fd
= -1.
librdkafka will maintain a copy of the payload
.
RD_EXPORT void rd_kafka_queue_cb_event_enable | ( | rd_kafka_queue_t * | rkqu, |
void(*)(rd_kafka_t *rk, void *qev_opaque) | event_cb, | ||
void * | qev_opaque | ||
) |
Enable callback event triggering for queue.
The callback will be called from an internal librdkafka thread when a new element is enqueued on a previously empty queue.
To remove event triggering call with event_cb
= NULL.
The qev_opaque
is passed to the callback's qev_opaque
argument.
RD_EXPORT void rd_kafka_queue_yield | ( | rd_kafka_queue_t * | rkqu | ) |
Cancels the current rd_kafka_queue_poll() on rkqu
.
An application may use this from another thread to force an immediate return to the calling code (caller of rd_kafka_queue_poll()). Must not be used from signal handlers since that may cause deadlocks.
RD_EXPORT int rd_kafka_consume_start | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int64_t | offset | ||
) |
Start consuming messages for topic rkt
and partition
at offset offset
which may either be an absolute (0..N) or one of the logical offsets:
rdkafka will attempt to keep queued.min.messages
(config property) messages in the local queue by repeatedly fetching batches of messages from the broker until the threshold is reached.
The application shall use one of the rd_kafka_consume*()
functions to consume messages from the local queue, each kafka message being represented as a rd_kafka_message_t *
object.
rd_kafka_consume_start()
must not be called multiple times for the same topic and partition without stopping consumption first with rd_kafka_consume_stop()
.
partition
is invalid. (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)Use rd_kafka_errno2err()
to convert sytem errno
to rd_kafka_resp_err_t
RD_EXPORT int rd_kafka_consume_start_queue | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int64_t | offset, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Same as rd_kafka_consume_start() but re-routes incoming messages to the provided queue rkqu
(which must have been previously allocated with rd_kafka_queue_new()
.
The application must use one of the rd_kafka_consume_*_queue()
functions to receive fetched messages.
rd_kafka_consume_start_queue()
must not be called multiple times for the same topic and partition without stopping consumption first with rd_kafka_consume_stop()
. rd_kafka_consume_start()
and rd_kafka_consume_start_queue()
must not be combined for the same topic and partition.
RD_EXPORT int rd_kafka_consume_stop | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition | ||
) |
Stop consuming messages for topic rkt
and partition
, purging all messages currently in the local queue.
NOTE: To enforce synchronisation this call will block until the internal fetcher has terminated and offsets are committed to configured storage method.
The application needs to be stop all consumers before calling rd_kafka_destroy()
on the main object handle.
errno
). RD_EXPORT rd_kafka_resp_err_t rd_kafka_seek | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int64_t | offset, | ||
int | timeout_ms | ||
) |
Seek consumer for topic+partition to offset
which is either an absolute or logical offset.
If timeout_ms
is specified (not 0) the seek call will wait this long for the consumer to update its fetcher state for the given partition with the new offset. This guarantees that no previously fetched messages for the old offset (or fetch position) will be passed to the application.
If the timeout is reached the internal state will be unknown to the caller and this function returns RD_KAFKA_RESP_ERR__TIMED_OUT
.
If timeout_ms
is 0 it will initiate the seek but return immediately without any error reporting (e.g., async).
This call will purge all pre-fetched messages for the given partition, which may be up to queued.max.message.kbytes
in size. Repeated use of seek may thus lead to increased network usage as messages are re-fetched from the broker.
RD_KAFKA_RESP_ERR__NO_ERROR
on success else an error code.RD_EXPORT rd_kafka_error_t* rd_kafka_seek_partitions | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | partitions, | ||
int | timeout_ms | ||
) |
Seek consumer for partitions in partitions
to the per-partition offset in the .offset field of
partitions
.
The offset may be either absolute (>= 0) or a logical offset.
If timeout_ms
is specified (not 0) the seek call will wait this long for the consumer to update its fetcher state for the given partition with the new offset. This guarantees that no previously fetched messages for the old offset (or fetch position) will be passed to the application.
If the timeout is reached the internal state will be unknown to the caller and this function returns RD_KAFKA_RESP_ERR__TIMED_OUT
.
If timeout_ms
is 0 it will initiate the seek but return immediately without any error reporting (e.g., async).
This call will purge all pre-fetched messages for the given partition, which may be up to queued.max.message.kbytes
in size. Repeated use of seek may thus lead to increased network usage as messages are re-fetched from the broker.
Individual partition errors are reported in the per-partition .err field of
partitions
.
RD_EXPORT rd_kafka_message_t* rd_kafka_consume | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int | timeout_ms | ||
) |
Consume a single message from topic rkt
and partition
.
timeout_ms
is maximum amount of time to wait for a message to be received. Consumer must have been previously started with rd_kafka_consume_start()
.
NULL
on error. The message object must be destroyed with rd_kafka_message_destroy()
when the application is done with it.Errors (when returning NULL):
timeout_ms
was reached with no new messages fetched.rkt
+ partition
is unknown. (no prior rd_kafka_consume_start()
call)NOTE: The returned message's ..->err must be checked for errors. NOTE:
..->err
==
RD_KAFKA_RESP_ERR__PARTITION_EOF
signals that the end of the partition has been reached, which should typically not be considered an error. The application should handle this case (e.g., ignore).
RD_EXPORT ssize_t rd_kafka_consume_batch | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int | timeout_ms, | ||
rd_kafka_message_t ** | rkmessages, | ||
size_t | rkmessages_size | ||
) |
Consume up to rkmessages_size
from topic rkt
and partition
putting a pointer to each message in the application provided array rkmessages
(of size rkmessages_size
entries).
rd_kafka_consume_batch()
provides higher throughput performance than rd_kafka_consume()
.
timeout_ms
is the maximum amount of time to wait for all of rkmessages_size
messages to be put into rkmessages
. If no messages were available within the timeout period this function returns 0 and rkmessages
remains untouched. This differs somewhat from rd_kafka_consume()
.
The message objects must be destroyed with rd_kafka_message_destroy()
when the application is done with it.
rkmessages
, or -1 on error (same error codes as for rd_kafka_consume()
.RD_EXPORT int rd_kafka_consume_callback | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int | timeout_ms, | ||
void(*)(rd_kafka_message_t *rkmessage, void *commit_opaque) | consume_cb, | ||
void * | commit_opaque | ||
) |
Consumes messages from topic rkt
and partition
, calling the provided callback for each consumed messsage.
rd_kafka_consume_callback()
provides higher throughput performance than both rd_kafka_consume()
and rd_kafka_consume_batch()
.
timeout_ms
is the maximum amount of time to wait for one or more messages to arrive.
The provided consume_cb
function is called for each message, the application MUST NOT call rd_kafka_message_destroy()
on the provided rkmessage
.
The commit_opaque
argument is passed to the consume_cb
as commit_opaque
.
RD_EXPORT rd_kafka_message_t* rd_kafka_consume_queue | ( | rd_kafka_queue_t * | rkqu, |
int | timeout_ms | ||
) |
Consume from queue.
RD_EXPORT ssize_t rd_kafka_consume_batch_queue | ( | rd_kafka_queue_t * | rkqu, |
int | timeout_ms, | ||
rd_kafka_message_t ** | rkmessages, | ||
size_t | rkmessages_size | ||
) |
Consume batch of messages from queue.
RD_EXPORT int rd_kafka_consume_callback_queue | ( | rd_kafka_queue_t * | rkqu, |
int | timeout_ms, | ||
void(*)(rd_kafka_message_t *rkmessage, void *commit_opaque) | consume_cb, | ||
void * | commit_opaque | ||
) |
Consume multiple messages from queue with callback.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_offset_store | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int64_t | offset | ||
) |
Store offset offset
+ 1 for topic rkt
partition partition
.
The offset
+ 1 will be committed (written) to broker (or file) according to
or manual offset-less commit()auto.commit.interval.ms
enable.auto.offset.store
must be set to "false" when using this API.RD_EXPORT rd_kafka_resp_err_t rd_kafka_offsets_store | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | offsets | ||
) |
Store offsets for next auto-commit for one or more partitions.
The offset will be committed (written) to the offset store according to
or manual offset-less commit().auto.commit.interval.ms
Per-partition success/error status propagated through each partition's .err for all return values (even NO_ERROR) except INVALID_ARG.
.offset field is stored as is, it will NOT be + 1.enable.auto.offset.store
must be set to "false" when using this API.enable.auto.offset.store
is true, or RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION or RD_KAFKA_RESP_ERR__STATE if none of the offsets could be stored. RD_EXPORT rd_kafka_error_t* rd_kafka_offset_store_message | ( | rd_kafka_message_t * | rkmessage | ) |
Store offset +1 for the consumed message.
The message offset + 1 will be committed to broker according to
or manual offset-less commit()auto.commit.interval.ms
enable.auto.offset.store
must be set to "false" when using this API.RD_EXPORT rd_kafka_resp_err_t rd_kafka_subscribe | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | topics | ||
) |
Subscribe to topic set using balanced consumer groups.
Wildcard (regex) topics are supported: any topic name in the topics
list that is prefixed with "^"
will be regex-matched to the full list of topics in the cluster and matching topics will be added to the subscription list.
The full topic list is retrieved every topic.metadata.refresh.interval.ms
to pick up new or delete topics that match the subscription. If there is any change to the matched topics the consumer will immediately rejoin the group with the updated set of subscribed topics.
Regex and full topic names can be mixed in topics
.
.topic field is used in the supplied topics
list, all other fields are ignored.session.timeout.ms
* 2 or more to complete.topics
. The error will be RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART for non-existent topics, and RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED for unauthorized topics. The consumer error will be raised through rd_kafka_consumer_poll() (et.al.) with the rd_kafka_message_t.err
field set to one of the error codes mentioned above. The subscribe function itself is asynchronous and will not return an error on unavailable topics.RD_EXPORT rd_kafka_resp_err_t rd_kafka_subscription | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t ** | topics | ||
) |
Returns the current topic subscription.
topic
is updated to point to a newly allocated topic list (possibly empty).RD_EXPORT rd_kafka_message_t* rd_kafka_consumer_poll | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Poll the consumer for messages or events.
Will block for at most timeout_ms
milliseconds.
->err
is RD_KAFKA_RESP_ERR_NO_ERROR, or an event or error for any other value.max.poll.interval.ms
to remain a member of the consumer group.Noteworthy errors returned in ->err:
max.poll.interval.ms
.RD_EXPORT rd_kafka_resp_err_t rd_kafka_consumer_close | ( | rd_kafka_t * | rk | ) |
Close the consumer.
This call will block until the consumer has revoked its assignment, calling the rebalance_cb
if it is configured, committed offsets to broker, and left the consumer group (if applicable). The maximum blocking time is roughly limited to session.timeout.ms.
RD_EXPORT rd_kafka_error_t* rd_kafka_consumer_close_queue | ( | rd_kafka_t * | rk, |
rd_kafka_queue_t * | rkqu | ||
) |
Asynchronously close the consumer.
Performs the same actions as rd_kafka_consumer_close() but in a background thread.
Rebalance events/callbacks (etc) will be forwarded to the application-provided rkqu
. The application must poll/serve this queue until rd_kafka_consumer_closed() returns true.
rkqu
.RD_EXPORT int rd_kafka_consumer_closed | ( | rd_kafka_t * | rk | ) |
Should be used in conjunction with rd_kafka_consumer_close_queue() to know when the consumer has been closed.
RD_EXPORT rd_kafka_error_t* rd_kafka_incremental_assign | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Incrementally add partitions
to the current assignment.
If a COOPERATIVE assignor (i.e. incremental rebalancing) is being used, this method should be used in a rebalance callback to adjust the current assignment appropriately in the case where the rebalance type is RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS. The application must pass the partition list passed to the callback (or a copy of it), even if the list is empty. partitions
must not be NULL. This method may also be used outside the context of a rebalance callback.
RD_EXPORT rd_kafka_error_t* rd_kafka_incremental_unassign | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Incrementally remove partitions
from the current assignment.
If a COOPERATIVE assignor (i.e. incremental rebalancing) is being used, this method should be used in a rebalance callback to adjust the current assignment appropriately in the case where the rebalance type is RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS. The application must pass the partition list passed to the callback (or a copy of it), even if the list is empty. partitions
must not be NULL. This method may also be used outside the context of a rebalance callback.
const RD_EXPORT char* rd_kafka_rebalance_protocol | ( | rd_kafka_t * | rk | ) |
The rebalance protocol currently in use. This will be "NONE" if the consumer has not (yet) joined a group, else it will match the rebalance protocol ("EAGER", "COOPERATIVE") of the configured and selected assignor(s). All configured assignors must have the same protocol type, meaning online migration of a consumer group from using one protocol to another (in particular upgading from EAGER to COOPERATIVE) without a restart is not currently supported.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_assign | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Atomic assignment of partitions to consume.
The new partitions
will replace the existing assignment.
A zero-length partitions
will treat the partitions as a valid, albeit empty assignment, and maintain internal state, while a NULL
value for partitions
will reset and clear the internal state.
When used from a rebalance callback, the application should pass the partition list passed to the callback (or a copy of it) even if the list is empty (i.e. should not pass NULL in this case) so as to maintain internal join state. This is not strictly required - the application may adjust the assignment provided by the group. However, this is rarely useful in practice.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_assignment | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t ** | partitions | ||
) |
Returns the current partition assignment as set by rd_kafka_assign() or rd_kafka_incremental_assign().
partitions
is updated to point to a newly allocated partition list (possibly empty).RD_EXPORT int rd_kafka_assignment_lost | ( | rd_kafka_t * | rk | ) |
Check whether the consumer considers the current assignment to have been lost involuntarily. This method is only applicable for use with a high level subscribing consumer. Assignments are revoked immediately when determined to have been lost, so this method is only useful when reacting to a RD_KAFKA_EVENT_REBALANCE event or from within a rebalance_cb. Partitions that have been lost may already be owned by other members in the group and therefore commiting offsets, for example, may fail.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | offsets, | ||
int | async | ||
) |
Commit offsets on broker for the provided list of partitions.
offsets
should contain topic
, partition
, offset
and possibly metadata
. The offset
should be the offset where consumption will resume, i.e., the last processed offset + 1. If offsets
is NULL the current partition assignment will be used instead.
If async
is false this operation will block until the broker offset commit is done, returning the resulting success or error code.
If a rd_kafka_conf_set_offset_commit_cb() offset commit callback has been configured the callback will be enqueued for a future call to rd_kafka_poll(), rd_kafka_consumer_poll() or similar.
FIXME: Update below documentation.
RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH is returned, when using group.protocol=consumer
, if the commit failed because the member has switched to a new member epoch. This error code can be retried. Partition level error is also set in the offsets
.
RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID is returned, when using group.protocol=consumer
, if the member has been removed from the consumer group This error code is permanent, uncommitted messages will be reprocessed by this or a different member and committed there. Partition level error is also set in the offsets
.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit_message | ( | rd_kafka_t * | rk, |
const rd_kafka_message_t * | rkmessage, | ||
int | async | ||
) |
Commit message's offset on broker for the message's partition. The committed offset is the message's offset + 1.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_commit_queue | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | offsets, | ||
rd_kafka_queue_t * | rkqu, | ||
void(*)(rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *commit_opaque) | cb, | ||
void * | commit_opaque | ||
) |
Commit offsets on broker for the provided list of partitions.
See rd_kafka_commit for offsets
semantics.
The result of the offset commit will be posted on the provided rkqu
queue.
If the application uses one of the poll APIs (rd_kafka_poll(), rd_kafka_consumer_poll(), rd_kafka_queue_poll(), ..) to serve the queue the cb
callback is required.
The commit_opaque
argument is passed to the callback as commit_opaque
, or if using the event API the callback is ignored and the offset commit result will be returned as an RD_KAFKA_EVENT_COMMIT event and the commit_opaque
value will be available with rd_kafka_event_opaque().
If rkqu
is NULL a temporary queue will be created and the callback will be served by this call.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_committed | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | partitions, | ||
int | timeout_ms | ||
) |
Retrieve committed offsets for topics+partitions.
The offset
field of each requested partition will either be set to stored offset or to RD_KAFKA_OFFSET_INVALID in case there was no stored offset for that partition.
Committed offsets will be returned according to the isolation.level
configuration property, if set to read_committed
(default) then only stable offsets for fully committed transactions will be returned, while read_uncommitted
may return offsets for not yet committed transactions.
offset
or err
field of each partitions'
element is filled in with the stored offset, or a partition specific error. Else returns an error code. RD_EXPORT rd_kafka_resp_err_t rd_kafka_position | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | partitions | ||
) |
Retrieve current positions (offsets) for topics+partitions.
The offset
field of each requested partition will be set to the offset of the last consumed message + 1, or RD_KAFKA_OFFSET_INVALID in case there was no previous message.
offset
or err
field of each partitions'
element is filled in with the stored offset, or a partition specific error. Else returns an error code. RD_EXPORT rd_kafka_consumer_group_metadata_t* rd_kafka_consumer_group_metadata | ( | rd_kafka_t * | rk | ) |
rk
is not a consumer configured with a group.id
. This metadata object should be passed to the transactional producer's rd_kafka_send_offsets_to_transaction() API.RD_EXPORT rd_kafka_consumer_group_metadata_t* rd_kafka_consumer_group_metadata_new | ( | const char * | group_id | ) |
Create a new consumer group metadata object. This is typically only used for writing tests.
group_id | The group id. |
RD_EXPORT rd_kafka_consumer_group_metadata_t* rd_kafka_consumer_group_metadata_new_with_genid | ( | const char * | group_id, |
int32_t | generation_id, | ||
const char * | member_id, | ||
const char * | group_instance_id | ||
) |
Create a new consumer group metadata object. This is typically only used for writing tests.
group_id | The group id. |
generation_id | The group generation id. |
member_id | The group member id. |
group_instance_id | The group instance id (may be NULL). |
const RD_EXPORT char* rd_kafka_consumer_group_metadata_member_id | ( | const rd_kafka_consumer_group_metadata_t * | group_metadata | ) |
Get member id of a group metadata.
group_metadata | The group metadata |
group_metadata
.group_metadata
. RD_EXPORT rd_kafka_error_t* rd_kafka_consumer_group_metadata_write | ( | const rd_kafka_consumer_group_metadata_t * | cgmd, |
void ** | bufferp, | ||
size_t * | sizep | ||
) |
Serialize the consumer group metadata to a binary format. This is mainly for client binding use and not for application use.
cgmd | Metadata to be serialized. |
bufferp | On success this pointer will be updated to point to na allocated buffer containing the serialized metadata. The buffer must be freed with rd_kafka_mem_free(). |
sizep | The pointed to size will be updated with the size of the serialized buffer. |
RD_EXPORT rd_kafka_error_t* rd_kafka_consumer_group_metadata_read | ( | rd_kafka_consumer_group_metadata_t ** | cgmdp, |
const void * | buffer, | ||
size_t | size | ||
) |
Reads serialized consumer group metadata and returns a consumer group metadata object. This is mainly for client binding use and not for application use.
cgmdp | On success this pointer will be updated to point to a new consumer group metadata object which must be freed with rd_kafka_consumer_group_metadata_destroy(). |
buffer | Pointer to the serialized data. |
size | Size of the serialized data. |
RD_EXPORT int rd_kafka_produce | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int | msgflags, | ||
void * | payload, | ||
size_t | len, | ||
const void * | key, | ||
size_t | keylen, | ||
void * | msg_opaque | ||
) |
Produce and send a single message to broker.
rkt
is the target topic which must have been previously created with rd_kafka_topic_new()
.
rd_kafka_produce()
is an asynchronous non-blocking API. See rd_kafka_conf_set_dr_msg_cb
on how to setup a callback to be called once the delivery status (success or failure) is known. The delivery report is triggered by the application calling rd_kafka_poll()
(at regular intervals) or rd_kafka_flush()
(at termination).
Since producing is asynchronous, you should call rd_kafka_flush()
before you destroy the producer. Otherwise, any outstanding messages will be silently discarded.
When temporary errors occur, librdkafka automatically retries to produce the messages. Retries are triggered after retry.backoff.ms and when the leader broker for the given partition is available. Otherwise, librdkafka falls back to polling the topic metadata to monitor when a new leader is elected (see the topic.metadata.refresh.fast.interval.ms and topic.metadata.refresh.interval.ms configurations) and then performs a retry. A delivery error will occur if the message could not be produced within message.timeout.ms.
See the "Message reliability" chapter in INTRODUCTION.md for more information.
partition
is the target partition, either:
msgflags
is zero or more of the following flags OR:ed together: RD_KAFKA_MSG_F_BLOCK - block produce*
() call if queue.buffering.max.messages
or queue.buffering.max.kbytes
are exceeded. Messages are considered in-queue from the point they are accepted by produce() until their corresponding delivery report callback/event returns. It is thus a requirement to call rd_kafka_poll() (or equiv.) from a separate thread when F_BLOCK is used. See WARNING on RD_KAFKA_MSG_F_BLOCK
above.
RD_KAFKA_MSG_F_FREE - rdkafka will free(3) payload
when it is done with it. RD_KAFKA_MSG_F_COPY - the payload
data will be copied and the payload
pointer will not be used by rdkafka after the call returns. RD_KAFKA_MSG_F_PARTITION - produce_batch() will honour per-message partition, either set manually or by the configured partitioner.
.._F_FREE and .._F_COPY are mutually exclusive. If neither of these are set, the caller must ensure that the memory backing payload
remains valid and is not modified or reused until the delivery callback is invoked. Other buffers passed to rd_kafka_produce()
don't have this restriction on reuse, i.e. the memory backing the key or the topic name may be reused as soon as rd_kafka_produce()
returns.
If the function returns -1 and RD_KAFKA_MSG_F_FREE was specified, then the memory associated with the payload is still the caller's responsibility.
payload
is the message payload of size len
bytes.
key
is an optional message key of size keylen
bytes, if non-NULL it will be passed to the topic partitioner as well as be sent with the message to the broker and passed on to the consumer.
msg_opaque
is an optional application-provided per-message opaque pointer that will provided in the message's delivery report callback (dr_msg_cb
or dr_cb
) and the rd_kafka_message_t
_private
field.
transactional.id
is configured) producing is only allowed during an on-going transaction, namely after rd_kafka_begin_transaction() has been called.partition
is unknown in the Kafka cluster. (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)errno
to rdkafka error code. RD_EXPORT rd_kafka_resp_err_t rd_kafka_producev | ( | rd_kafka_t * | rk, |
... | |||
) |
Produce and send a single message to broker.
The message is defined by a va-arg list using rd_kafka_vtype_t
tag tuples which must be terminated with a single RD_KAFKA_V_END
.
RD_KAFKA_RESP_ERR_NO_ERROR
on success, else an error code as described in rd_kafka_produce(). RD_KAFKA_RESP_ERR__CONFLICT
is returned if _V_HEADER and _V_HEADERS are mixed.RD_EXPORT rd_kafka_error_t* rd_kafka_produceva | ( | rd_kafka_t * | rk, |
const rd_kafka_vu_t * | vus, | ||
size_t | cnt | ||
) |
Produce and send a single message to broker.
The message is defined by an array of rd_kafka_vu_t
of count cnt
.
RD_EXPORT int rd_kafka_produce_batch | ( | rd_kafka_topic_t * | rkt, |
int32_t | partition, | ||
int | msgflags, | ||
rd_kafka_message_t * | rkmessages, | ||
int | message_cnt | ||
) |
Produce multiple messages.
If partition is RD_KAFKA_PARTITION_UA the configured partitioner will be run for each message (slower), otherwise the messages will be enqueued to the specified partition directly (faster).
The messages are provided in the array rkmessages
of count message_cnt
elements. The partition
and msgflags
are used for all provided messages.
Honoured rkmessages
[] fields are:
message_cnt
.RD_KAFKA_MSG_F_PARTITION
is set in msgflags
, the
.partition field of the rkmessages
is used instead of partition
.rkmessages
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_flush | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Wait until all outstanding produce requests, et.al, are completed. This should typically be done prior to destroying a producer instance to make sure all queued and in-flight produce requests are completed before terminating.
linger.ms
time will be ignored for the duration of the call, queued messages will be sent to the broker as soon as possible.timeout_ms
was reached before all outstanding requests were completed, else RD_KAFKA_RESP_ERR_NO_ERRORRD_EXPORT rd_kafka_resp_err_t rd_kafka_purge | ( | rd_kafka_t * | rk, |
int | purge_flags | ||
) |
Purge messages currently handled by the producer instance.
rk | Client instance. |
purge_flags | Tells which messages to purge and how. |
The application will need to call rd_kafka_poll() or rd_kafka_flush() afterwards to serve the delivery report callbacks of the purged messages.
Messages purged from internal queues fail with the delivery report error code set to RD_KAFKA_RESP_ERR__PURGE_QUEUE, while purged messages that are in-flight to or from the broker will fail with the error code set to RD_KAFKA_RESP_ERR__PURGE_INFLIGHT.
purge
flags are invalid or unknown, RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED if called on a non-producer client instance. RD_EXPORT rd_kafka_resp_err_t rd_kafka_metadata | ( | rd_kafka_t * | rk, |
int | all_topics, | ||
rd_kafka_topic_t * | only_rkt, | ||
const struct rd_kafka_metadata ** | metadatap, | ||
int | timeout_ms | ||
) |
Request Metadata from broker.
Parameters:
all_topics
if non-zero: request info about all topics in cluster, if zero: only request info about locally known topics.only_rkt
only request info about this topicmetadatap
pointer to hold metadata result. The *metadatap
pointer must be released with rd_kafka_metadata_destroy().timeout_ms
maximum response time before failing.all_topics
is non-zero the Metadata response information may trigger a re-join if any subscribed topics have changed partition count or existence state.RD_EXPORT int rd_kafka_Node_id | ( | const rd_kafka_Node_t * | node | ) |
Get the id of node
.
node | The Node instance. |
const RD_EXPORT char* rd_kafka_Node_host | ( | const rd_kafka_Node_t * | node | ) |
Get the host of node
.
node | The Node instance. |
node
object. RD_EXPORT uint16_t rd_kafka_Node_port | ( | const rd_kafka_Node_t * | node | ) |
Get the port of node
.
node | The Node instance. |
const RD_EXPORT char* rd_kafka_Node_rack | ( | const rd_kafka_Node_t * | node | ) |
Get the rack of node
.
node | The Node instance |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_list_groups | ( | rd_kafka_t * | rk, |
const char * | group, | ||
const struct rd_kafka_group_list ** | grplistp, | ||
int | timeout_ms | ||
) |
List and describe client groups in cluster.
group
is an optional group name to describe, otherwise (NULL
) all groups are returned.
timeout_ms
is the (approximate) maximum time to wait for response from brokers and must be a positive value.
RD_KAFKA_RESP_ERR__NO_ERROR
on success and grplistp
is updated to point to a newly allocated list of groups. RD_KAFKA_RESP_ERR__PARTIAL
if not all brokers responded in time but at least one group is returned in grplistlp
. RD_KAFKA_RESP_ERR__TIMED_OUT
if no groups were returned in the given timeframe but not all brokers have yet responded, or if the list of brokers in the cluster could not be obtained within the given timeframe. RD_KAFKA_RESP_ERR__TRANSPORT
if no brokers were found. Other error codes may also be returned from the request layer.The grplistp
remains untouched if any error code is returned, with the exception of RD_KAFKA_RESP_ERR__PARTIAL which behaves as RD_KAFKA_RESP_ERR__NO_ERROR (success) but with an incomplete group list.
const RD_EXPORT char* rd_kafka_consumer_group_state_name | ( | rd_kafka_consumer_group_state_t | state | ) |
Returns a name for a state code.
state | The state value. |
RD_EXPORT rd_kafka_consumer_group_state_t rd_kafka_consumer_group_state_code | ( | const char * | name | ) |
Returns a code for a state name.
name | The state name. |
const RD_EXPORT char* rd_kafka_consumer_group_type_name | ( | rd_kafka_consumer_group_type_t | type | ) |
Returns a name for a group type code.
type | The group type value. |
RD_EXPORT rd_kafka_consumer_group_type_t rd_kafka_consumer_group_type_code | ( | const char * | name | ) |
Returns a code for a group type name.
name | The group type name. |
RD_EXPORT int rd_kafka_brokers_add | ( | rd_kafka_t * | rk, |
const char * | brokerlist | ||
) |
Adds one or more brokers to the kafka handle's list of initial bootstrap brokers.
Additional brokers will be discovered automatically as soon as rdkafka connects to a broker by querying the broker metadata.
If a broker name resolves to multiple addresses (and possibly address families) all will be used for connection attempts in round-robin fashion.
brokerlist
is a ,-separated list of brokers in the format: <broker1>
,<broker2>,.. Where each broker is in either the host or URL based format: <host>
[:<port>] <proto>
://<host>[:port] <proto>
is either PLAINTEXT
, SSL
, SASL
, SASL_PLAINTEXT
The two formats can be mixed but ultimately the value of the security.protocol
config property decides what brokers are allowed.
Example: brokerlist = "broker1:10000,broker2" brokerlist = "SSL://broker3:9000,ssl://broker2"
metadata.broker.list
or bootstrap.servers
configuration property (preferred method).bootstrap.servers
configuration property. RD_EXPORT RD_DEPRECATED void rd_kafka_set_logger | ( | rd_kafka_t * | rk, |
void(*)(const rd_kafka_t *rk, int level, const char *fac, const char *buf) | func | ||
) |
Set logger function.
The default is to print to stderr, but a syslog logger is also available, see rd_kafka_log_(print|syslog) for the builtin alternatives. Alternatively the application may provide its own logger callback. Or pass 'func' as NULL to disable logging.
rk
may be passed as NULL in the callback. RD_EXPORT void rd_kafka_set_log_level | ( | rd_kafka_t * | rk, |
int | level | ||
) |
Specifies the maximum logging level emitted by internal kafka logging and debugging.
"log_level"
configuration property instead."debug"
configuration property is set the log level is automatically adjusted to LOG_DEBUG
(7). RD_EXPORT void rd_kafka_log_syslog | ( | const rd_kafka_t * | rk, |
int | level, | ||
const char * | fac, | ||
const char * | buf | ||
) |
Builtin log sink: print to syslog.
RD_EXPORT int rd_kafka_outq_len | ( | rd_kafka_t * | rk | ) |
Returns the current out queue length.
The out queue length is the sum of:
An application should wait for the return value of this function to reach zero before terminating to make sure outstanding messages, requests (such as offset commits), callbacks and events are fully processed. See rd_kafka_flush().
RD_EXPORT void rd_kafka_dump | ( | FILE * | fp, |
rd_kafka_t * | rk | ||
) |
Dumps rdkafka's internal state for handle rk
to stream fp
.
This is only useful for debugging rdkafka, showing state and statistics for brokers, topics, partitions, etc.
RD_EXPORT int rd_kafka_thread_cnt | ( | void | ) |
Retrieve the current number of threads in use by librdkafka.
Used by regression tests.
RD_EXPORT int rd_kafka_wait_destroyed | ( | int | timeout_ms | ) |
Wait for all rd_kafka_t objects to be destroyed.
Returns 0 if all kafka objects are now destroyed, or -1 if the timeout was reached.
RD_EXPORT int rd_kafka_unittest | ( | void | ) |
Run librdkafka's built-in unit-tests.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_poll_set_consumer | ( | rd_kafka_t * | rk | ) |
Redirect the main (rd_kafka_poll()) queue to the KafkaConsumer's queue (rd_kafka_consumer_poll()).
RD_EXPORT rd_kafka_event_type_t rd_kafka_event_type | ( | const rd_kafka_event_t * | rkev | ) |
rkev
as NULL in which case RD_KAFKA_EVENT_NONE is returned. const RD_EXPORT char* rd_kafka_event_name | ( | const rd_kafka_event_t * | rkev | ) |
rkev
as NULL in which case the name for RD_KAFKA_EVENT_NONE is returned. RD_EXPORT void rd_kafka_event_destroy | ( | rd_kafka_event_t * | rkev | ) |
Destroy an event.
rkev
as NULL in which case no action is performed. const RD_EXPORT rd_kafka_message_t* rd_kafka_event_message_next | ( | rd_kafka_event_t * | rkev | ) |
Call repeatedly until it returns NULL.
Event types:
RD_EXPORT size_t rd_kafka_event_message_array | ( | rd_kafka_event_t * | rkev, |
const rd_kafka_message_t ** | rkmessages, | ||
size_t | size | ||
) |
Extacts size
message(s) from the event into the pre-allocated array rkmessages
.
Event types:
RD_EXPORT size_t rd_kafka_event_message_count | ( | rd_kafka_event_t * | rkev | ) |
Event types:
const RD_EXPORT char* rd_kafka_event_config_string | ( | rd_kafka_event_t * | rkev | ) |
The returned memory is read-only and its lifetime is the same as the event object.
Event types:
RD_EXPORT rd_kafka_resp_err_t rd_kafka_event_error | ( | rd_kafka_event_t * | rkev | ) |
Use rd_kafka_event_error_is_fatal() to detect if this is a fatal error.
Event types:
const RD_EXPORT char* rd_kafka_event_error_string | ( | rd_kafka_event_t * | rkev | ) |
Event types:
RD_EXPORT int rd_kafka_event_error_is_fatal | ( | rd_kafka_event_t * | rkev | ) |
Event types:
RD_EXPORT void* rd_kafka_event_opaque | ( | rd_kafka_event_t * | rkev | ) |
Event types:
RD_EXPORT int rd_kafka_event_log | ( | rd_kafka_event_t * | rkev, |
const char ** | fac, | ||
const char ** | str, | ||
int * | level | ||
) |
Extract log message from the event.
Event types:
RD_EXPORT int rd_kafka_event_debug_contexts | ( | rd_kafka_event_t * | rkev, |
char * | dst, | ||
size_t | dstsize | ||
) |
Extract log debug context from event.
Event types:
rkev | the event to extract data from. |
dst | destination string for comma separated list. |
dstsize | size of provided dst buffer. |
const RD_EXPORT char* rd_kafka_event_stats | ( | rd_kafka_event_t * | rkev | ) |
Extract stats from the event.
Event types:
RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_event_topic_partition_list | ( | rd_kafka_event_t * | rkev | ) |
Event types:
RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_event_topic_partition | ( | rd_kafka_event_t * | rkev | ) |
Event types: RD_KAFKA_EVENT_ERROR (for partition level errors)
const RD_EXPORT rd_kafka_CreateTopics_result_t* rd_kafka_event_CreateTopics_result | ( | rd_kafka_event_t * | rkev | ) |
Get CreateTopics result.
Event types: RD_KAFKA_EVENT_CREATETOPICS_RESULT
const RD_EXPORT rd_kafka_DeleteTopics_result_t* rd_kafka_event_DeleteTopics_result | ( | rd_kafka_event_t * | rkev | ) |
Get DeleteTopics result.
Event types: RD_KAFKA_EVENT_DELETETOPICS_RESULT
const RD_EXPORT rd_kafka_CreatePartitions_result_t* rd_kafka_event_CreatePartitions_result | ( | rd_kafka_event_t * | rkev | ) |
Get CreatePartitions result.
Event types: RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT
const RD_EXPORT rd_kafka_AlterConfigs_result_t* rd_kafka_event_AlterConfigs_result | ( | rd_kafka_event_t * | rkev | ) |
Get AlterConfigs result.
Event types: RD_KAFKA_EVENT_ALTERCONFIGS_RESULT
const RD_EXPORT rd_kafka_IncrementalAlterConfigs_result_t* rd_kafka_event_IncrementalAlterConfigs_result | ( | rd_kafka_event_t * | rkev | ) |
Get IncrementalAlterConfigs result.
Event types: RD_KAFKA_EVENT_INCREMENTALALTERCONFIGS_RESULT
const RD_EXPORT rd_kafka_DescribeConfigs_result_t* rd_kafka_event_DescribeConfigs_result | ( | rd_kafka_event_t * | rkev | ) |
Get DescribeConfigs result.
Event types: RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT
const RD_EXPORT rd_kafka_DeleteRecords_result_t* rd_kafka_event_DeleteRecords_result | ( | rd_kafka_event_t * | rkev | ) |
Event types: RD_KAFKA_EVENT_DELETERECORDS_RESULT
const RD_EXPORT rd_kafka_ListConsumerGroups_result_t* rd_kafka_event_ListConsumerGroups_result | ( | rd_kafka_event_t * | rkev | ) |
Get ListConsumerGroups result.
rkev
object.Event types: RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT
const RD_EXPORT rd_kafka_DescribeConsumerGroups_result_t* rd_kafka_event_DescribeConsumerGroups_result | ( | rd_kafka_event_t * | rkev | ) |
Get DescribeConsumerGroups result.
rkev
object.Event types: RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT
const RD_EXPORT rd_kafka_DescribeTopics_result_t* rd_kafka_event_DescribeTopics_result | ( | rd_kafka_event_t * | rkev | ) |
Get DescribeTopics result.
rkev
object.Event types: RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT
const RD_EXPORT rd_kafka_DescribeCluster_result_t* rd_kafka_event_DescribeCluster_result | ( | rd_kafka_event_t * | rkev | ) |
Get DescribeCluster result.
rkev
object.Event types: RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT
const RD_EXPORT rd_kafka_DeleteGroups_result_t* rd_kafka_event_DeleteGroups_result | ( | rd_kafka_event_t * | rkev | ) |
Get DeleteGroups result.
Event types: RD_KAFKA_EVENT_DELETEGROUPS_RESULT
const RD_EXPORT rd_kafka_DeleteConsumerGroupOffsets_result_t* rd_kafka_event_DeleteConsumerGroupOffsets_result | ( | rd_kafka_event_t * | rkev | ) |
Get DeleteConsumerGroupOffsets result.
Event types: RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_CreateAcls_result_t* rd_kafka_event_CreateAcls_result | ( | rd_kafka_event_t * | rkev | ) |
Event types: RD_KAFKA_EVENT_CREATEACLS_RESULT
const RD_EXPORT rd_kafka_DescribeAcls_result_t* rd_kafka_event_DescribeAcls_result | ( | rd_kafka_event_t * | rkev | ) |
Event types: RD_KAFKA_EVENT_DESCRIBEACLS_RESULT
const RD_EXPORT rd_kafka_DeleteAcls_result_t* rd_kafka_event_DeleteAcls_result | ( | rd_kafka_event_t * | rkev | ) |
Event types: RD_KAFKA_EVENT_DELETEACLS_RESULT
const RD_EXPORT rd_kafka_ListConsumerGroupOffsets_result_t* rd_kafka_event_ListConsumerGroupOffsets_result | ( | rd_kafka_event_t * | rkev | ) |
Get ListConsumerGroupOffsets result.
rkev
object.Event types: RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_AlterConsumerGroupOffsets_result_t* rd_kafka_event_AlterConsumerGroupOffsets_result | ( | rd_kafka_event_t * | rkev | ) |
Get AlterConsumerGroupOffsets result.
rkev
object.Event types: RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_ListOffsets_result_t* rd_kafka_event_ListOffsets_result | ( | rd_kafka_event_t * | rkev | ) |
Get ListOffsets result.
rkev
object.Event types: RD_KAFKA_EVENT_LISTOFFSETS_RESULT
const RD_EXPORT rd_kafka_DescribeUserScramCredentials_result_t* rd_kafka_event_DescribeUserScramCredentials_result | ( | rd_kafka_event_t * | rkev | ) |
Get DescribeUserScramCredentials result.
rkev
object.Event types: RD_KAFKA_EVENT_DESCRIBEUSERSCRAMCREDENTIALS_RESULT
const RD_EXPORT rd_kafka_AlterUserScramCredentials_result_t* rd_kafka_event_AlterUserScramCredentials_result | ( | rd_kafka_event_t * | rkev | ) |
Get AlterUserScramCredentials result.
rkev
object.Event types: RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT
const RD_EXPORT rd_kafka_ElectLeaders_result_t* rd_kafka_event_ElectLeaders_result | ( | rd_kafka_event_t * | rkev | ) |
Get ElectLeaders result.
rkev
object.Event types: RD_KAFKA_EVENT_ELECTLEADERS_RESULT
RD_EXPORT rd_kafka_event_t* rd_kafka_queue_poll | ( | rd_kafka_queue_t * | rkqu, |
int | timeout_ms | ||
) |
Poll a queue for an event for max timeout_ms
.
RD_EXPORT int rd_kafka_queue_poll_callback | ( | rd_kafka_queue_t * | rkqu, |
int | timeout_ms | ||
) |
Poll a queue for events served through callbacks for max timeout_ms
.
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_set | ( | rd_kafka_conf_t * | conf, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_conf_set_t * | on_conf_set, | ||
void * | ic_opaque | ||
) |
Append an on_conf_set() interceptor.
conf | Configuration object. |
ic_name | Interceptor name, used in logging. |
on_conf_set | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_dup | ( | rd_kafka_conf_t * | conf, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_conf_dup_t * | on_conf_dup, | ||
void * | ic_opaque | ||
) |
Append an on_conf_dup() interceptor.
conf | Configuration object. |
ic_name | Interceptor name, used in logging. |
on_conf_dup | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_conf_destroy | ( | rd_kafka_conf_t * | conf, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_conf_destroy_t * | on_conf_destroy, | ||
void * | ic_opaque | ||
) |
Append an on_conf_destroy() interceptor.
conf | Configuration object. |
ic_name | Interceptor name, used in logging. |
on_conf_destroy | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_conf_interceptor_add_on_new | ( | rd_kafka_conf_t * | conf, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_new_t * | on_new, | ||
void * | ic_opaque | ||
) |
Append an on_new() interceptor.
conf | Configuration object. |
ic_name | Interceptor name, used in logging. |
on_new | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_destroy | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_destroy_t * | on_destroy, | ||
void * | ic_opaque | ||
) |
Append an on_destroy() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_destroy | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_send | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_send_t * | on_send, | ||
void * | ic_opaque | ||
) |
Append an on_send() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_send | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_acknowledgement | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_acknowledgement_t * | on_acknowledgement, | ||
void * | ic_opaque | ||
) |
Append an on_acknowledgement() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_acknowledgement | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_consume | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_consume_t * | on_consume, | ||
void * | ic_opaque | ||
) |
Append an on_consume() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_consume | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_commit | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_commit_t * | on_commit, | ||
void * | ic_opaque | ||
) |
Append an on_commit() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_commit() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_request_sent | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_request_sent_t * | on_request_sent, | ||
void * | ic_opaque | ||
) |
Append an on_request_sent() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_request_sent() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_response_received | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_response_received_t * | on_response_received, | ||
void * | ic_opaque | ||
) |
Append an on_response_received() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_response_received() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_thread_start | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_thread_start_t * | on_thread_start, | ||
void * | ic_opaque | ||
) |
Append an on_thread_start() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_thread_start() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_thread_exit | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_thread_exit_t * | on_thread_exit, | ||
void * | ic_opaque | ||
) |
Append an on_thread_exit() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_thread_exit() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_interceptor_add_on_broker_state_change | ( | rd_kafka_t * | rk, |
const char * | ic_name, | ||
rd_kafka_interceptor_f_on_broker_state_change_t * | on_broker_state_change, | ||
void * | ic_opaque | ||
) |
Append an on_broker_state_change() interceptor.
rk | Client instance. |
ic_name | Interceptor name, used in logging. |
on_broker_state_change() | Function pointer. |
ic_opaque | Opaque value that will be passed to the function. |
ic_name
and function has already been added to conf
. RD_EXPORT rd_kafka_resp_err_t rd_kafka_topic_result_error | ( | const rd_kafka_topic_result_t * | topicres | ) |
Topic result provides per-topic operation result information.
const RD_EXPORT char* rd_kafka_topic_result_error_string | ( | const rd_kafka_topic_result_t * | topicres | ) |
topicres
. const RD_EXPORT char* rd_kafka_topic_result_name | ( | const rd_kafka_topic_result_t * | topicres | ) |
topicres
. const RD_EXPORT rd_kafka_error_t* rd_kafka_group_result_error | ( | const rd_kafka_group_result_t * | groupres | ) |
Group result provides per-group operation result information.
groupres
. const RD_EXPORT char* rd_kafka_group_result_name | ( | const rd_kafka_group_result_t * | groupres | ) |
groupres
. const RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_group_result_partitions | ( | const rd_kafka_group_result_t * | groupres | ) |
groupres
. const RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_topic_partition_result_partition | ( | const rd_kafka_topic_partition_result_t * | partition_result | ) |
Topic Partition Result provides per-topic+partition operation result Consists of TopicPartition object and error object.
partition_result
. The error object is set inside the topic partition object. For the detailed error information, use rd_kafka_topic_partition_result_error() const RD_EXPORT rd_kafka_error_t* rd_kafka_topic_partition_result_error | ( | const rd_kafka_topic_partition_result_t * | partition_result | ) |
partition_result
. RD_EXPORT rd_kafka_AdminOptions_t* rd_kafka_AdminOptions_new | ( | rd_kafka_t * | rk, |
rd_kafka_admin_op_t | for_api | ||
) |
Create a new AdminOptions object.
The options object is not modified by the Admin API request APIs, (e.g. CreateTopics) and may be reused for multiple calls.
rk | Client instance. |
for_api | Specifies what Admin API this AdminOptions object will be used for, which will enforce what AdminOptions_set_..() calls may be used based on the API, causing unsupported set..() calls to fail. Specifying RD_KAFKA_ADMIN_OP_ANY disables the enforcement allowing any option to be set, even if the option is not used in a future call to an Admin API method. |
for_api
was set to an unknown API op type. RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_request_timeout | ( | rd_kafka_AdminOptions_t * | options, |
int | timeout_ms, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Sets the overall request timeout, including broker lookup, request transmission, operation time on broker, and response.
options | Admin options. |
timeout_ms | Timeout in milliseconds. Defaults to socket.timeout.ms . |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
errstr
.RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_operation_timeout | ( | rd_kafka_AdminOptions_t * | options, |
int | timeout_ms, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Sets the broker's operation timeout, such as the timeout for CreateTopics to complete the creation of topics on the controller before returning a result to the application.
CreateTopics: values <= 0 will return immediately after triggering topic creation, while > 0 will wait this long for topic creation to propagate in cluster. Default: 60 seconds.
DeleteTopics: same semantics as CreateTopics. CreatePartitions: same semantics as CreateTopics.
options | Admin options. |
timeout_ms | Timeout in milliseconds. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
errstr
.RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_validate_only | ( | rd_kafka_AdminOptions_t * | options, |
int | true_or_false, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Tell broker to only validate the request, without performing the requested operation (create topics, etc).
options | Admin options. |
true_or_false | Defaults to false. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
errstr
.RD_EXPORT rd_kafka_resp_err_t rd_kafka_AdminOptions_set_broker | ( | rd_kafka_AdminOptions_t * | options, |
int32_t | broker_id, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Override what broker the Admin request will be sent to.
By default, Admin requests are sent to the controller broker, with the following exceptions:
options | Admin Options. |
broker_id | The broker to send the request to. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
errstr
.RD_EXPORT rd_kafka_error_t* rd_kafka_AdminOptions_set_require_stable_offsets | ( | rd_kafka_AdminOptions_t * | options, |
int | true_or_false | ||
) |
Whether broker should return stable offsets (transaction-committed).
options | Admin options. |
true_or_false | Defaults to false. |
RD_EXPORT rd_kafka_error_t* rd_kafka_AdminOptions_set_include_authorized_operations | ( | rd_kafka_AdminOptions_t * | options, |
int | true_or_false | ||
) |
Whether broker should return authorized operations for the given resource in the DescribeConsumerGroups, DescribeTopics, or DescribeCluster calls.
options | Admin options. |
true_or_false | Defaults to false. |
RD_EXPORT rd_kafka_error_t* rd_kafka_AdminOptions_set_match_consumer_group_states | ( | rd_kafka_AdminOptions_t * | options, |
const rd_kafka_consumer_group_state_t * | consumer_group_states, | ||
size_t | consumer_group_states_cnt | ||
) |
Set consumer groups states to query for.
options | Admin options. |
consumer_group_states | Array of consumer group states. |
consumer_group_states_cnt | Size of the consumer_group_states array. |
RD_EXPORT rd_kafka_error_t* rd_kafka_AdminOptions_set_match_consumer_group_types | ( | rd_kafka_AdminOptions_t * | options, |
const rd_kafka_consumer_group_type_t * | consumer_group_types, | ||
size_t | consumer_group_types_cnt | ||
) |
Set consumer groups types to query for.
options | Admin options. |
consumer_group_types | Array of consumer group types. |
consumer_group_types_cnt | Size of the consumer_group_types array. |
RD_EXPORT rd_kafka_NewTopic_t* rd_kafka_NewTopic_new | ( | const char * | topic, |
int | num_partitions, | ||
int | replication_factor, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Create a new NewTopic object. This object is later passed to rd_kafka_CreateTopics().
topic | Topic name to create. |
num_partitions | Number of partitions in topic, or -1 to use the broker's default partition count (>= 2.4.0). |
replication_factor | Default replication factor for the topic's partitions, or -1 to use the broker's default replication factor (>= 2.4.0) or if set_replica_assignment() will be used. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewTopic_set_replica_assignment | ( | rd_kafka_NewTopic_t * | new_topic, |
int32_t | partition, | ||
int32_t * | broker_ids, | ||
size_t | broker_id_cnt, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Set the replica (broker) assignment for partition
to the replica set in broker_ids
(of broker_id_cnt
elements).
replication_factor
of -1.RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewTopic_set_config | ( | rd_kafka_NewTopic_t * | new_topic, |
const char * | name, | ||
const char * | value | ||
) |
Set (broker-side) topic configuration name/value pair.
RD_EXPORT void rd_kafka_CreateTopics | ( | rd_kafka_t * | rk, |
rd_kafka_NewTopic_t ** | new_topics, | ||
size_t | new_topic_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Create topics in cluster as specified by the new_topics
array of size new_topic_cnt
elements.
rk | Client instance. |
new_topics | Array of new topics to create. |
new_topic_cnt | Number of elements in new_topics array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_EVENT_CREATETOPICS_RESULT
const RD_EXPORT rd_kafka_topic_result_t** rd_kafka_CreateTopics_result_topics | ( | const rd_kafka_CreateTopics_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of topic results from a CreateTopics result.
The returned topics
life-time is the same as the result
object.
result | Result to get topics from. |
cntp | Updated to the number of elements in the array. |
RD_EXPORT rd_kafka_DeleteTopic_t* rd_kafka_DeleteTopic_new | ( | const char * | topic | ) |
Create a new DeleteTopic object. This object is later passed to rd_kafka_DeleteTopics().
topic | Topic name to delete. |
RD_EXPORT void rd_kafka_DeleteTopics | ( | rd_kafka_t * | rk, |
rd_kafka_DeleteTopic_t ** | del_topics, | ||
size_t | del_topic_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Delete topics from cluster as specified by the topics
array of size topic_cnt
elements.
rk | Client instance. |
del_topics | Array of topics to delete. |
del_topic_cnt | Number of elements in topics array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DELETETOPICS_RESULT
const RD_EXPORT rd_kafka_topic_result_t** rd_kafka_DeleteTopics_result_topics | ( | const rd_kafka_DeleteTopics_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of topic results from a DeleteTopics result.
The returned topics
life-time is the same as the result
object.
result | Result to get topic results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT rd_kafka_NewPartitions_t* rd_kafka_NewPartitions_new | ( | const char * | topic, |
size_t | new_total_cnt, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Create a new NewPartitions. This object is later passed to rd_kafka_CreatePartitions() to increase the number of partitions to new_total_cnt
for an existing topic.
topic | Topic name to create more partitions for. |
new_total_cnt | Increase the topic's partition count to this value. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_NewPartitions_set_replica_assignment | ( | rd_kafka_NewPartitions_t * | new_parts, |
int32_t | new_partition_idx, | ||
int32_t * | broker_ids, | ||
size_t | broker_id_cnt, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Set the replica (broker id) assignment for new_partition_idx
to the replica set in broker_ids
(of broker_id_cnt
elements).
new_partition_idx
0 is the first new partition, 1 is the second, and so on.broker_id_cnt
should match the topic's replication factor.RD_EXPORT void rd_kafka_CreatePartitions | ( | rd_kafka_t * | rk, |
rd_kafka_NewPartitions_t ** | new_parts, | ||
size_t | new_parts_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Create additional partitions for the given topics, as specified by the new_parts
array of size new_parts_cnt
elements.
rk | Client instance. |
new_parts | Array of topics for which new partitions are to be created. |
new_parts_cnt | Number of elements in new_parts array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT
const RD_EXPORT rd_kafka_topic_result_t** rd_kafka_CreatePartitions_result_topics | ( | const rd_kafka_CreatePartitions_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of topic results from a CreatePartitions result.
The returned topics
life-time is the same as the result
object.
result | Result o get topic results from. |
cntp | is updated to the number of elements in the array. |
const RD_EXPORT char* rd_kafka_ConfigSource_name | ( | rd_kafka_ConfigSource_t | confsource | ) |
confsource
. const RD_EXPORT char* rd_kafka_ConfigEntry_name | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
const RD_EXPORT char* rd_kafka_ConfigEntry_value | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
RD_EXPORT rd_kafka_ConfigSource_t rd_kafka_ConfigEntry_source | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
RD_EXPORT int rd_kafka_ConfigEntry_is_read_only | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
RD_EXPORT int rd_kafka_ConfigEntry_is_default | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
RD_EXPORT int rd_kafka_ConfigEntry_is_sensitive | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
RD_EXPORT int rd_kafka_ConfigEntry_is_synonym | ( | const rd_kafka_ConfigEntry_t * | entry | ) |
const RD_EXPORT rd_kafka_ConfigEntry_t** rd_kafka_ConfigEntry_synonyms | ( | const rd_kafka_ConfigEntry_t * | entry, |
size_t * | cntp | ||
) |
entry | Entry to get synonyms for. |
cntp | is updated to the number of elements in the array. |
conf
. const RD_EXPORT char* rd_kafka_ResourcePatternType_name | ( | rd_kafka_ResourcePatternType_t | resource_pattern_type | ) |
resource_pattern_type
const RD_EXPORT char* rd_kafka_ResourceType_name | ( | rd_kafka_ResourceType_t | restype | ) |
restype
RD_EXPORT rd_kafka_ConfigResource_t* rd_kafka_ConfigResource_new | ( | rd_kafka_ResourceType_t | restype, |
const char * | resname | ||
) |
Create new ConfigResource object.
restype | The resource type (e.g., RD_KAFKA_RESOURCE_TOPIC) |
resname | The resource name (e.g., the topic name) |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_ConfigResource_set_config | ( | rd_kafka_ConfigResource_t * | config, |
const char * | name, | ||
const char * | value | ||
) |
Set configuration name value pair.
config | ConfigResource to set config property on. |
name | Configuration name, depends on resource type. |
value | Configuration value, depends on resource type and name . Set to NULL to revert configuration value to default. |
This will overwrite the current value.
RD_EXPORT rd_kafka_error_t* rd_kafka_ConfigResource_add_incremental_config | ( | rd_kafka_ConfigResource_t * | config, |
const char * | name, | ||
rd_kafka_AlterConfigOpType_t | op_type, | ||
const char * | value | ||
) |
Add the value of the configuration entry for a subsequent incremental alter config operation. APPEND and SUBTRACT are possible for list-type configuration entries only.
config | ConfigResource to add config property to. |
name | Configuration name, depends on resource type. |
op_type | Operation type, one of rd_kafka_AlterConfigOpType_t. |
value | Configuration value, depends on resource type and name . Set to NULL , only with with op_type set to DELETE, to revert configuration value to default. |
const RD_EXPORT rd_kafka_ConfigEntry_t** rd_kafka_ConfigResource_configs | ( | const rd_kafka_ConfigResource_t * | config, |
size_t * | cntp | ||
) |
Get an array of config entries from a ConfigResource object.
The returned object life-times are the same as the config
object.
config | ConfigResource to get configs from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT rd_kafka_ResourceType_t rd_kafka_ConfigResource_type | ( | const rd_kafka_ConfigResource_t * | config | ) |
config
const RD_EXPORT char* rd_kafka_ConfigResource_name | ( | const rd_kafka_ConfigResource_t * | config | ) |
config
RD_EXPORT rd_kafka_resp_err_t rd_kafka_ConfigResource_error | ( | const rd_kafka_ConfigResource_t * | config | ) |
const RD_EXPORT char* rd_kafka_ConfigResource_error_string | ( | const rd_kafka_ConfigResource_t * | config | ) |
RD_EXPORT void rd_kafka_AlterConfigs | ( | rd_kafka_t * | rk, |
rd_kafka_ConfigResource_t ** | configs, | ||
size_t | config_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Update the configuration for the specified resources. Updates are not transactional so they may succeed for a subset of the provided resources while the others fail. The configuration for a particular resource is updated atomically, replacing values using the provided ConfigEntrys and reverting unspecified ConfigEntrys to their default values.
RD_KAFKA_RESOURCE_BROKER
is allowed per call since these resource requests must be sent to the broker specified in the resource.const RD_EXPORT rd_kafka_ConfigResource_t** rd_kafka_AlterConfigs_result_resources | ( | const rd_kafka_AlterConfigs_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of resource results from a AlterConfigs result.
Use rd_kafka_ConfigResource_error()
and rd_kafka_ConfigResource_error_string()
to extract per-resource error results on the returned array elements.
The returned object life-times are the same as the result
object.
result | Result object to get resource results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_IncrementalAlterConfigs | ( | rd_kafka_t * | rk, |
rd_kafka_ConfigResource_t ** | configs, | ||
size_t | config_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Incrementally update the configuration for the specified resources. Updates are not transactional so they may succeed for some resources while fail for others. The configs for a particular resource are updated atomically, executing the corresponding incremental operations on the provided configurations.
RD_KAFKA_RESOURCE_BROKER
is allowed per call since these resource requests must be sent to the broker specified in the resource. Broker option will be ignored in this case.rk | Client instance. |
configs | Array of config entries to alter. |
config_cnt | Number of elements in configs array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
const RD_EXPORT rd_kafka_ConfigResource_t** rd_kafka_IncrementalAlterConfigs_result_resources | ( | const rd_kafka_IncrementalAlterConfigs_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of resource results from a IncrementalAlterConfigs result.
Use rd_kafka_ConfigResource_error()
and rd_kafka_ConfigResource_error_string()
to extract per-resource error results on the returned array elements.
The returned object life-times are the same as the result
object.
result | Result object to get resource results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_DescribeConfigs | ( | rd_kafka_t * | rk, |
rd_kafka_ConfigResource_t ** | configs, | ||
size_t | config_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Get configuration for the specified resources in configs
.
The returned configuration includes default values and the rd_kafka_ConfigEntry_is_default() or rd_kafka_ConfigEntry_source() methods may be used to distinguish them from user supplied values.
The value of config entries where rd_kafka_ConfigEntry_is_sensitive() is true will always be NULL to avoid disclosing sensitive information, such as security settings.
Configuration entries where rd_kafka_ConfigEntry_is_read_only() is true can't be updated (with rd_kafka_AlterConfigs()).
Synonym configuration entries are returned if the broker supports it (broker version >= 1.1.0). See rd_kafka_ConfigEntry_synonyms().
RD_KAFKA_RESOURCE_BROKER
is allowed per call since these resource requests must be sent to the broker specified in the resource. const RD_EXPORT rd_kafka_ConfigResource_t** rd_kafka_DescribeConfigs_result_resources | ( | const rd_kafka_DescribeConfigs_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of resource results from a DescribeConfigs result.
The returned resources
life-time is the same as the result
object.
result | Result object to get resource results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT rd_kafka_DeleteRecords_t* rd_kafka_DeleteRecords_new | ( | const rd_kafka_topic_partition_list_t * | before_offsets | ) |
Create a new DeleteRecords object. This object is later passed to rd_kafka_DeleteRecords().
before_offsets
must contain topic
, partition
, and offset
is the offset before which the messages will be deleted (exclusive). Set offset
to RD_KAFKA_OFFSET_END (high-watermark) in order to delete all data in the partition.
before_offsets | For each partition delete all messages up to but not including the specified offset. |
RD_EXPORT void rd_kafka_DeleteRecords | ( | rd_kafka_t * | rk, |
rd_kafka_DeleteRecords_t ** | del_records, | ||
size_t | del_record_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Delete records (messages) in topic partitions older than the offsets provided.
rk | Client instance. |
del_records | The offsets to delete (up to). Currently only one DeleteRecords_t (but containing multiple offsets) is supported. |
del_record_cnt | The number of elements in del_records, must be 1. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
rdkafka
will wait for the request to complete.RD_KAFKA_EVENT_DELETERECORDS_RESULT
const RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_DeleteRecords_result_offsets | ( | const rd_kafka_DeleteRecords_result_t * | result | ) |
Get a list of topic and partition results from a DeleteRecords result. The returned objects will contain topic
, partition
, offset
and err
. offset
will be set to the post-deletion low-watermark (smallest available offset of all live replicas). err
will be set per-partition if deletion failed.
The returned object's life-time is the same as the result
object.
RD_EXPORT rd_kafka_TopicCollection_t* rd_kafka_TopicCollection_of_topic_names | ( | const char ** | topics, |
size_t | topics_cnt | ||
) |
Creates a new TopicCollection for passing to rd_kafka_DescribeTopics.
topics | A list of topics. |
topics_cnt | Count of topics. |
RD_EXPORT void rd_kafka_DescribeTopics | ( | rd_kafka_t * | rk, |
const rd_kafka_TopicCollection_t * | topics, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Describe topics as specified by the topics
array of size topics_cnt
elements.
rk | Client instance. |
topics | Collection of topics to describe. |
options | Optional admin options, or NULL for defaults. Valid options:
|
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT
const RD_EXPORT rd_kafka_TopicDescription_t** rd_kafka_DescribeTopics_result_topics | ( | const rd_kafka_DescribeTopics_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of topic results from a DescribeTopics result.
result | Result to get topics results from. |
cntp | is updated to the number of elements in the array. |
result
object. const RD_EXPORT rd_kafka_TopicPartitionInfo_t** rd_kafka_TopicDescription_partitions | ( | const rd_kafka_TopicDescription_t * | topicdesc, |
size_t * | cntp | ||
) |
Gets an array of partitions for the topicdesc
topic.
topicdesc | The topic description. |
cntp | is updated to the number of partitions in the array. |
topicdesc
object. const RD_EXPORT int rd_kafka_TopicPartitionInfo_partition | ( | const rd_kafka_TopicPartitionInfo_t * | partition | ) |
Gets the partition id for partition
.
partition | The partition info. |
const RD_EXPORT rd_kafka_Node_t* rd_kafka_TopicPartitionInfo_leader | ( | const rd_kafka_TopicPartitionInfo_t * | partition | ) |
Gets the partition leader for partition
.
partition | The partition info. |
partition
object. const RD_EXPORT rd_kafka_Node_t** rd_kafka_TopicPartitionInfo_isr | ( | const rd_kafka_TopicPartitionInfo_t * | partition, |
size_t * | cntp | ||
) |
Gets the partition in-sync replicas for partition
.
partition | The partition info. |
cntp | is updated with in-sync replicas count. |
partition
object. const RD_EXPORT rd_kafka_Node_t** rd_kafka_TopicPartitionInfo_replicas | ( | const rd_kafka_TopicPartitionInfo_t * | partition, |
size_t * | cntp | ||
) |
Gets the partition replicas for partition
.
partition | The partition info. |
cntp | is updated with partition replicas count. |
partition
object. const RD_EXPORT rd_kafka_AclOperation_t* rd_kafka_TopicDescription_authorized_operations | ( | const rd_kafka_TopicDescription_t * | topicdesc, |
size_t * | cntp | ||
) |
Gets the topic authorized ACL operations for the topicdesc
topic.
topicdesc | The topic description. |
cntp | is updated with authorized ACL operations count. |
topicdesc
object. const RD_EXPORT char* rd_kafka_TopicDescription_name | ( | const rd_kafka_TopicDescription_t * | topicdesc | ) |
Gets the topic name for the topicdesc
topic.
topicdesc | The topic description. |
topicdesc
object. const RD_EXPORT rd_kafka_Uuid_t* rd_kafka_TopicDescription_topic_id | ( | const rd_kafka_TopicDescription_t * | topicdesc | ) |
Gets the topic id for the topicdesc
topic.
topicdesc | The topic description. |
topicdesc
object. RD_EXPORT int rd_kafka_TopicDescription_is_internal | ( | const rd_kafka_TopicDescription_t * | topicdesc | ) |
Gets if the topicdesc
topic is internal.
topicdesc | The topic description. |
const RD_EXPORT rd_kafka_error_t* rd_kafka_TopicDescription_error | ( | const rd_kafka_TopicDescription_t * | topicdesc | ) |
Gets the error for the topicdesc
topic.
topicdesc | The topic description. |
topicdesc
object. RD_EXPORT void rd_kafka_DescribeCluster | ( | rd_kafka_t * | rk, |
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Describes the cluster.
rk | Client instance. |
options | Optional admin options, or NULL for defaults. Valid options:
|
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT
const RD_EXPORT rd_kafka_Node_t** rd_kafka_DescribeCluster_result_nodes | ( | const rd_kafka_DescribeCluster_result_t * | result, |
size_t * | cntp | ||
) |
Gets the broker nodes for the result
cluster.
result | The result of DescribeCluster. |
cntp | is updated with the count of broker nodes. |
result
object. const RD_EXPORT rd_kafka_AclOperation_t* rd_kafka_DescribeCluster_result_authorized_operations | ( | const rd_kafka_DescribeCluster_result_t * | result, |
size_t * | cntp | ||
) |
Gets the authorized ACL operations for the result
cluster.
result | The result of DescribeCluster. |
cntp | is updated with authorized ACL operations count. |
result
object. const RD_EXPORT rd_kafka_Node_t* rd_kafka_DescribeCluster_result_controller | ( | const rd_kafka_DescribeCluster_result_t * | result | ) |
Gets the current controller for the result
cluster.
result | The result of DescribeCluster. |
const RD_EXPORT char* rd_kafka_DescribeCluster_result_cluster_id | ( | const rd_kafka_DescribeCluster_result_t * | result | ) |
Gets the cluster id for the result
cluster.
result | The result of DescribeCluster. |
result
object. RD_EXPORT void rd_kafka_ListConsumerGroups | ( | rd_kafka_t * | rk, |
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
List the consumer groups available in the cluster.
rk | Client instance. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT
const RD_EXPORT char* rd_kafka_ConsumerGroupListing_group_id | ( | const rd_kafka_ConsumerGroupListing_t * | grplist | ) |
Gets the group id for the grplist
group.
grplist | The group listing. |
grplist
object. RD_EXPORT int rd_kafka_ConsumerGroupListing_is_simple_consumer_group | ( | const rd_kafka_ConsumerGroupListing_t * | grplist | ) |
Is the grplist
group a simple consumer group.
grplist | The group listing. |
RD_EXPORT rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupListing_state | ( | const rd_kafka_ConsumerGroupListing_t * | grplist | ) |
Gets state for the grplist
group.
grplist | The group listing. |
RD_EXPORT rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupListing_type | ( | const rd_kafka_ConsumerGroupListing_t * | grplist | ) |
Gets type for the grplist
group.
grplist | The group listing. |
const RD_EXPORT rd_kafka_ConsumerGroupListing_t** rd_kafka_ListConsumerGroups_result_valid | ( | const rd_kafka_ListConsumerGroups_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of valid list groups from a ListConsumerGroups result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
result
object. const RD_EXPORT rd_kafka_error_t** rd_kafka_ListConsumerGroups_result_errors | ( | const rd_kafka_ListConsumerGroups_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of errors from a ListConsumerGroups call result.
The returned errors life-time is the same as the result
object.
result | ListConsumerGroups result. |
cntp | Is updated to the number of elements in the array. |
result
.result
object. RD_EXPORT void rd_kafka_DescribeConsumerGroups | ( | rd_kafka_t * | rk, |
const char ** | groups, | ||
size_t | groups_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Describe groups from cluster as specified by the groups
array of size groups_cnt
elements.
rk | Client instance. |
groups | Array of groups to describe. |
groups_cnt | Number of elements in groups array. |
options | Optional admin options, or NULL for defaults. Valid options:
|
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT
const RD_EXPORT rd_kafka_ConsumerGroupDescription_t** rd_kafka_DescribeConsumerGroups_result_groups | ( | const rd_kafka_DescribeConsumerGroups_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of group results from a DescribeConsumerGroups result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
result
object. const RD_EXPORT char* rd_kafka_ConsumerGroupDescription_group_id | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets the group id for the grpdesc
group.
grpdesc | The group description. |
grpdesc
object. const RD_EXPORT rd_kafka_error_t* rd_kafka_ConsumerGroupDescription_error | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets the error for the grpdesc
group.
grpdesc | The group description. |
grpdesc
object. RD_EXPORT int rd_kafka_ConsumerGroupDescription_is_simple_consumer_group | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Is the grpdesc
group a simple consumer group.
grpdesc | The group description. |
const RD_EXPORT char* rd_kafka_ConsumerGroupDescription_partition_assignor | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets the partition assignor for the grpdesc
group.
grpdesc | The group description. |
grpdesc
object. const RD_EXPORT rd_kafka_AclOperation_t* rd_kafka_ConsumerGroupDescription_authorized_operations | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc, |
size_t * | cntp | ||
) |
Gets the authorized ACL operations for the grpdesc
group.
grpdesc | The group description. |
cntp | is updated with authorized ACL operations count. |
grpdesc
object. RD_EXPORT rd_kafka_consumer_group_state_t rd_kafka_ConsumerGroupDescription_state | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets state for the grpdesc
group.
grpdesc | The group description. |
const RD_EXPORT rd_kafka_Node_t* rd_kafka_ConsumerGroupDescription_coordinator | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets the coordinator for the grpdesc
group.
grpdesc | The group description. |
grpdesc
object. RD_EXPORT size_t rd_kafka_ConsumerGroupDescription_member_count | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc | ) |
Gets the members count of grpdesc
group.
grpdesc | The group description. |
const RD_EXPORT rd_kafka_MemberDescription_t* rd_kafka_ConsumerGroupDescription_member | ( | const rd_kafka_ConsumerGroupDescription_t * | grpdesc, |
size_t | idx | ||
) |
Gets a member of grpdesc
group.
grpdesc | The group description. |
idx | The member idx. |
idx
, or NULL if idx
is out of range.grpdesc
object. const RD_EXPORT char* rd_kafka_MemberDescription_client_id | ( | const rd_kafka_MemberDescription_t * | member | ) |
Gets client id of member
.
member | The group member. |
member
object. const RD_EXPORT char* rd_kafka_MemberDescription_group_instance_id | ( | const rd_kafka_MemberDescription_t * | member | ) |
Gets group instance id of member
.
member | The group member. |
member
object. const RD_EXPORT char* rd_kafka_MemberDescription_consumer_id | ( | const rd_kafka_MemberDescription_t * | member | ) |
Gets consumer id of member
.
member | The group member. |
member
object. const RD_EXPORT char* rd_kafka_MemberDescription_host | ( | const rd_kafka_MemberDescription_t * | member | ) |
Gets host of member
.
member | The group member. |
member
object. const RD_EXPORT rd_kafka_MemberAssignment_t* rd_kafka_MemberDescription_assignment | ( | const rd_kafka_MemberDescription_t * | member | ) |
Gets assignment of member
.
member | The group member. |
member
object. const RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_MemberAssignment_partitions | ( | const rd_kafka_MemberAssignment_t * | assignment | ) |
Gets assigned partitions of a member assignment
.
assignment | The group member assignment. |
assignment
object. RD_EXPORT rd_kafka_DeleteGroup_t* rd_kafka_DeleteGroup_new | ( | const char * | group | ) |
Create a new DeleteGroup object. This object is later passed to rd_kafka_DeleteGroups().
group | Name of group to delete. |
RD_EXPORT void rd_kafka_DeleteGroups | ( | rd_kafka_t * | rk, |
rd_kafka_DeleteGroup_t ** | del_groups, | ||
size_t | del_group_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Delete groups from cluster as specified by the del_groups
array of size del_group_cnt
elements.
rk | Client instance. |
del_groups | Array of groups to delete. |
del_group_cnt | Number of elements in del_groups array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DELETEGROUPS_RESULT
const RD_EXPORT rd_kafka_group_result_t** rd_kafka_DeleteGroups_result_groups | ( | const rd_kafka_DeleteGroups_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of group results from a DeleteGroups result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT rd_kafka_ListConsumerGroupOffsets_t* rd_kafka_ListConsumerGroupOffsets_new | ( | const char * | group_id, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Create a new ListConsumerGroupOffsets object. This object is later passed to rd_kafka_ListConsumerGroupOffsets().
group_id | Consumer group id. |
partitions | Partitions to list committed offsets for. Only the topic and partition fields are used. |
RD_EXPORT void rd_kafka_ListConsumerGroupOffsets | ( | rd_kafka_t * | rk, |
rd_kafka_ListConsumerGroupOffsets_t ** | list_grpoffsets, | ||
size_t | list_grpoffsets_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
List committed offsets for a set of partitions in a consumer group.
rk | Client instance. |
list_grpoffsets | Array of group committed offsets to list. MUST only be one single element. |
list_grpoffsets_cnt | Number of elements in list_grpoffsets array. MUST always be 1. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_LISTCONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_group_result_t** rd_kafka_ListConsumerGroupOffsets_result_groups | ( | const rd_kafka_ListConsumerGroupOffsets_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of results from a ListConsumerGroupOffsets result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
result
object. RD_EXPORT rd_kafka_AlterConsumerGroupOffsets_t* rd_kafka_AlterConsumerGroupOffsets_new | ( | const char * | group_id, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Create a new AlterConsumerGroupOffsets object. This object is later passed to rd_kafka_AlterConsumerGroupOffsets().
group_id | Consumer group id. |
partitions | Partitions to alter committed offsets for. Only the topic and partition fields are used. |
RD_EXPORT void rd_kafka_AlterConsumerGroupOffsets | ( | rd_kafka_t * | rk, |
rd_kafka_AlterConsumerGroupOffsets_t ** | alter_grpoffsets, | ||
size_t | alter_grpoffsets_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Alter committed offsets for a set of partitions in a consumer group. This will succeed at the partition level only if the group is not actively subscribed to the corresponding topic.
rk | Client instance. |
alter_grpoffsets | Array of group committed offsets to alter. MUST only be one single element. |
alter_grpoffsets_cnt | Number of elements in alter_grpoffsets array. MUST always be 1. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_ALTERCONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_group_result_t** rd_kafka_AlterConsumerGroupOffsets_result_groups | ( | const rd_kafka_AlterConsumerGroupOffsets_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of results from a AlterConsumerGroupOffsets result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
result
object. RD_EXPORT rd_kafka_DeleteConsumerGroupOffsets_t* rd_kafka_DeleteConsumerGroupOffsets_new | ( | const char * | group, |
const rd_kafka_topic_partition_list_t * | partitions | ||
) |
Create a new DeleteConsumerGroupOffsets object. This object is later passed to rd_kafka_DeleteConsumerGroupOffsets().
group | Consumer group id. |
partitions | Partitions to delete committed offsets for. Only the topic and partition fields are used. |
RD_EXPORT void rd_kafka_DeleteConsumerGroupOffsets | ( | rd_kafka_t * | rk, |
rd_kafka_DeleteConsumerGroupOffsets_t ** | del_grpoffsets, | ||
size_t | del_grpoffsets_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Delete committed offsets for a set of partitions in a consumer group. This will succeed at the partition level only if the group is not actively subscribed to the corresponding topic.
rk | Client instance. |
del_grpoffsets | Array of group committed offsets to delete. MUST only be one single element. |
del_grpoffsets_cnt | Number of elements in del_grpoffsets array. MUST always be 1. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_KAFKA_EVENT_DELETECONSUMERGROUPOFFSETS_RESULT
const RD_EXPORT rd_kafka_group_result_t** rd_kafka_DeleteConsumerGroupOffsets_result_groups | ( | const rd_kafka_DeleteConsumerGroupOffsets_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of results from a DeleteConsumerGroupOffsets result.
The returned groups life-time is the same as the result
object.
result | Result to get group results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_ListOffsets | ( | rd_kafka_t * | rk, |
rd_kafka_topic_partition_list_t * | topic_partitions, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
List offsets for the specified topic_partitions
. This operation enables to find the beginning offset, end offset as well as the offset matching a timestamp in partitions or the offset with max timestamp.
rk | Client instance. |
topic_partitions | topic_partition_list_t with the partitions and offsets to list. Each topic partition offset can be a value of the rd_kafka_OffsetSpec_t enum or a non-negative value, representing a timestamp, to query for the first offset after the given timestamp. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_ISOLATION_LEVEL_READ_UNCOMMITTED
RD_KAFKA_EVENT_LISTOFFSETS_RESULT
const RD_EXPORT rd_kafka_UserScramCredentialsDescription_t** rd_kafka_DescribeUserScramCredentials_result_descriptions | ( | const rd_kafka_DescribeUserScramCredentials_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of descriptions from a DescribeUserScramCredentials result.
The returned value life-time is the same as the result
object.
result | Result to get descriptions from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_DescribeUserScramCredentials | ( | rd_kafka_t * | rk, |
const char ** | users, | ||
size_t | user_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Describe SASL/SCRAM credentials. This operation is supported by brokers with version 2.7.0 or higher.
rk | Client instance. |
users | The users for which credentials are to be described. All users' credentials are described if NULL. |
user_cnt | Number of elements in users array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
RD_EXPORT rd_kafka_UserScramCredentialAlteration_t* rd_kafka_UserScramCredentialUpsertion_new | ( | const char * | username, |
rd_kafka_ScramMechanism_t | mechanism, | ||
int32_t | iterations, | ||
const unsigned char * | password, | ||
size_t | password_size, | ||
const unsigned char * | salt, | ||
size_t | salt_size | ||
) |
Allocates a new UserScramCredentialUpsertion given its fields. If salt isn't given a 64 B salt is generated using OpenSSL RAND_priv_bytes, if available.
username | The username (not empty). |
mechanism | SASL/SCRAM mechanism. |
iterations | SASL/SCRAM iterations. |
password | Password bytes (not empty). |
password_size | Size of password (greater than 0). |
salt | Salt bytes (optional). |
salt_size | Size of salt (optional). |
RD_EXPORT rd_kafka_UserScramCredentialAlteration_t* rd_kafka_UserScramCredentialDeletion_new | ( | const char * | username, |
rd_kafka_ScramMechanism_t | mechanism | ||
) |
Allocates a new UserScramCredentialDeletion given its fields.
username | The username (not empty). |
mechanism | SASL/SCRAM mechanism. |
const RD_EXPORT rd_kafka_AlterUserScramCredentials_result_response_t** rd_kafka_AlterUserScramCredentials_result_responses | ( | const rd_kafka_AlterUserScramCredentials_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of responses from a AlterUserScramCredentials result.
The returned value life-time is the same as the result
object.
result | Result to get responses from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_AlterUserScramCredentials | ( | rd_kafka_t * | rk, |
rd_kafka_UserScramCredentialAlteration_t ** | alterations, | ||
size_t | alteration_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Alter SASL/SCRAM credentials. This operation is supported by brokers with version 2.7.0 or higher.
rk | Client instance. |
alterations | The alterations to be applied. |
alteration_cnt | Number of elements in alterations array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
const RD_EXPORT rd_kafka_error_t* rd_kafka_acl_result_error | ( | const rd_kafka_acl_result_t * | aclres | ) |
const RD_EXPORT char* rd_kafka_AclOperation_name | ( | rd_kafka_AclOperation_t | acl_operation | ) |
acl_operation
const RD_EXPORT char* rd_kafka_AclPermissionType_name | ( | rd_kafka_AclPermissionType_t | acl_permission_type | ) |
acl_permission_type
RD_EXPORT rd_kafka_AclBinding_t* rd_kafka_AclBinding_new | ( | rd_kafka_ResourceType_t | restype, |
const char * | name, | ||
rd_kafka_ResourcePatternType_t | resource_pattern_type, | ||
const char * | principal, | ||
const char * | host, | ||
rd_kafka_AclOperation_t | operation, | ||
rd_kafka_AclPermissionType_t | permission_type, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Create a new AclBinding object. This object is later passed to rd_kafka_CreateAcls().
restype | The ResourceType. |
name | The resource name. |
resource_pattern_type | The pattern type. |
principal | A principal, following the kafka specification. |
host | An hostname or ip. |
operation | A Kafka operation. |
permission_type | A Kafka permission type. |
errstr | An error string for returning errors or NULL to not use it. |
errstr_size | The errstr size or 0 to not use it. |
RD_EXPORT rd_kafka_AclBindingFilter_t* rd_kafka_AclBindingFilter_new | ( | rd_kafka_ResourceType_t | restype, |
const char * | name, | ||
rd_kafka_ResourcePatternType_t | resource_pattern_type, | ||
const char * | principal, | ||
const char * | host, | ||
rd_kafka_AclOperation_t | operation, | ||
rd_kafka_AclPermissionType_t | permission_type, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Create a new AclBindingFilter object. This object is later passed to rd_kafka_DescribeAcls() or rd_kafka_DeletesAcls() in order to filter the acls to retrieve or to delete. Use the same rd_kafka_AclBinding functions to query or destroy it.
restype | The ResourceType or RD_KAFKA_RESOURCE_ANY if not filtering by this field. |
name | The resource name or NULL if not filtering by this field. |
resource_pattern_type | The pattern type or RD_KAFKA_RESOURCE_PATTERN_ANY if not filtering by this field. |
principal | A principal or NULL if not filtering by this field. |
host | An hostname or ip or NULL if not filtering by this field. |
operation | A Kafka operation or RD_KAFKA_ACL_OPERATION_ANY if not filtering by this field. |
permission_type | A Kafka permission type or RD_KAFKA_ACL_PERMISSION_TYPE_ANY if not filtering by this field. |
errstr | An error string for returning errors or NULL to not use it. |
errstr_size | The errstr size or 0 to not use it. |
RD_EXPORT rd_kafka_ResourceType_t rd_kafka_AclBinding_restype | ( | const rd_kafka_AclBinding_t * | acl | ) |
const RD_EXPORT char* rd_kafka_AclBinding_name | ( | const rd_kafka_AclBinding_t * | acl | ) |
acl
. const RD_EXPORT char* rd_kafka_AclBinding_principal | ( | const rd_kafka_AclBinding_t * | acl | ) |
acl
. const RD_EXPORT char* rd_kafka_AclBinding_host | ( | const rd_kafka_AclBinding_t * | acl | ) |
acl
. RD_EXPORT rd_kafka_AclOperation_t rd_kafka_AclBinding_operation | ( | const rd_kafka_AclBinding_t * | acl | ) |
RD_EXPORT rd_kafka_AclPermissionType_t rd_kafka_AclBinding_permission_type | ( | const rd_kafka_AclBinding_t * | acl | ) |
RD_EXPORT rd_kafka_ResourcePatternType_t rd_kafka_AclBinding_resource_pattern_type | ( | const rd_kafka_AclBinding_t * | acl | ) |
const RD_EXPORT rd_kafka_error_t* rd_kafka_AclBinding_error | ( | const rd_kafka_AclBinding_t * | acl | ) |
const RD_EXPORT rd_kafka_acl_result_t** rd_kafka_CreateAcls_result_acls | ( | const rd_kafka_CreateAcls_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of acl results from a CreateAcls result.
The returned acl
result life-time is the same as the result
object.
result | CreateAcls result to get acl results from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_CreateAcls | ( | rd_kafka_t * | rk, |
rd_kafka_AclBinding_t ** | new_acls, | ||
size_t | new_acls_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Create acls as specified by the new_acls
array of size new_topic_cnt
elements.
rk | Client instance. |
new_acls | Array of new acls to create. |
new_acls_cnt | Number of elements in new_acls array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_EVENT_CREATEACLS_RESULT
const RD_EXPORT rd_kafka_AclBinding_t** rd_kafka_DescribeAcls_result_acls | ( | const rd_kafka_DescribeAcls_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of resource results from a DescribeAcls result.
DescribeAcls - describe access control lists. The returned resources
life-time is the same as the result
object.
result | DescribeAcls result to get acls from. |
cntp | is updated to the number of elements in the array. |
RD_EXPORT void rd_kafka_DescribeAcls | ( | rd_kafka_t * | rk, |
rd_kafka_AclBindingFilter_t * | acl_filter, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Describe acls matching the filter provided in acl_filter
.
rk | Client instance. |
acl_filter | Filter for the returned acls. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_EVENT_DESCRIBEACLS_RESULT
const RD_EXPORT rd_kafka_DeleteAcls_result_response_t** rd_kafka_DeleteAcls_result_responses | ( | const rd_kafka_DeleteAcls_result_t * | result, |
size_t * | cntp | ||
) |
Get an array of DeleteAcls result responses from a DeleteAcls result.
The returned responses
life-time is the same as the result
object.
result | DeleteAcls result to get responses from. |
cntp | is updated to the number of elements in the array. |
const RD_EXPORT rd_kafka_error_t* rd_kafka_DeleteAcls_result_response_error | ( | const rd_kafka_DeleteAcls_result_response_t * | result_response | ) |
const RD_EXPORT rd_kafka_AclBinding_t** rd_kafka_DeleteAcls_result_response_matching_acls | ( | const rd_kafka_DeleteAcls_result_response_t * | result_response, |
size_t * | matching_acls_cntp | ||
) |
result_response
. RD_EXPORT void rd_kafka_DeleteAcls | ( | rd_kafka_t * | rk, |
rd_kafka_AclBindingFilter_t ** | del_acls, | ||
size_t | del_acls_cnt, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Delete acls matching the filteres provided in del_acls
array of size del_acls_cnt
.
rk | Client instance. |
del_acls | Filters for the acls to delete. |
del_acls_cnt | Number of elements in del_acls array. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
RD_KAFKA_EVENT_DELETEACLS_RESULT
RD_EXPORT rd_kafka_ElectLeaders_t* rd_kafka_ElectLeaders_new | ( | rd_kafka_ElectionType_t | election_type, |
rd_kafka_topic_partition_list_t * | partitions | ||
) |
Create a new rd_kafka_ElectLeaders_t object. This object is later passed to rd_kafka_ElectLeaders().
election_type | The election type that needs to be performed, preferred or unclean. |
partitions | The topic partitions for which the leader election needs to be performed. |
RD_EXPORT void rd_kafka_ElectLeaders_destroy | ( | rd_kafka_ElectLeaders_t * | elect_leaders | ) |
Destroy and free a rd_kafka_ElectLeaders_t object previously created with rd_kafka_ElectLeaders_new()
elect_leaders | The rd_kafka_ElectLeaders_t object to be destroyed. |
RD_EXPORT void rd_kafka_ElectLeaders | ( | rd_kafka_t * | rk, |
rd_kafka_ElectLeaders_t * | elect_leaders, | ||
const rd_kafka_AdminOptions_t * | options, | ||
rd_kafka_queue_t * | rkqu | ||
) |
Elect Leaders for the provided Topic Partitions according to the specified election type.
rk | Client instance. |
elect_leaders | The elect leaders request containing election type and partitions information. |
options | Optional admin options, or NULL for defaults. |
rkqu | Queue to emit result on. |
Supported admin options:
rdkafka
will wait for the request to complete.RD_KAFKA_EVENT_ELECTLEADERS_RESULT
const RD_EXPORT rd_kafka_topic_partition_result_t** rd_kafka_ElectLeaders_result_partitions | ( | const rd_kafka_ElectLeaders_result_t * | result, |
size_t * | cntp | ||
) |
Get the array of topic partition result objects from the elect leaders result event and populates the size of the array in cntp
.
result | The elect leaders result. |
cntp | The number of elements in the array. |
RD_EXPORT rd_kafka_resp_err_t rd_kafka_oauthbearer_set_token | ( | rd_kafka_t * | rk, |
const char * | token_value, | ||
int64_t | md_lifetime_ms, | ||
const char * | md_principal_name, | ||
const char ** | extensions, | ||
size_t | extension_size, | ||
char * | errstr, | ||
size_t | errstr_size | ||
) |
Set SASL/OAUTHBEARER token and metadata.
rk | Client instance. |
token_value | the mandatory token value to set, often (but not necessarily) a JWS compact serialization as per https://tools.ietf.org/html/rfc7515#section-3.1. |
md_lifetime_ms | when the token expires, in terms of the number of milliseconds since the epoch. |
md_principal_name | the mandatory Kafka principal name associated with the token. |
extensions | optional SASL extensions key-value array with extensions_size elements (number of keys * 2), where [i] is the key and [i+1] is the key's value, to be communicated to the broker as additional key-value pairs during the initial client response as per https://tools.ietf.org/html/rfc7628#section-3.1. The key-value pairs are copied. |
extension_size | the number of SASL extension keys plus values, which must be a non-negative multiple of 2. |
errstr | A human readable error string (nul-terminated) is written to this location that must be of at least errstr_size bytes. The errstr is only written in case of error. |
errstr_size | Writable size in errstr . |
The SASL/OAUTHBEARER token refresh callback or event handler should invoke this method upon success. The extension keys must not include the reserved key "`auth`", and all extension keys and values must conform to the required format as per https://tools.ietf.org/html/rfc7628#section-3.1:
key = 1*(ALPHA) value = *(VCHAR / SP / HTAB / CR / LF )
RD_KAFKA_RESP_ERR_NO_ERROR
on success, otherwise errstr
set and:RD_KAFKA_RESP_ERR__INVALID_ARG
if any of the arguments are invalid;RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED
if SASL/OAUTHBEARER is not supported by this build;RD_KAFKA_RESP_ERR__STATE
if SASL/OAUTHBEARER is supported but is not configured as the client's authentication mechanism.RD_EXPORT rd_kafka_resp_err_t rd_kafka_oauthbearer_set_token_failure | ( | rd_kafka_t * | rk, |
const char * | errstr | ||
) |
SASL/OAUTHBEARER token refresh failure indicator.
rk | Client instance. |
errstr | mandatory human readable error reason for failing to acquire a token. |
The SASL/OAUTHBEARER token refresh callback or event handler should invoke this method upon failure.
RD_KAFKA_RESP_ERR_NO_ERROR
on success, otherwise:RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED
if SASL/OAUTHBEARER is not supported by this build;RD_KAFKA_RESP_ERR__STATE
if SASL/OAUTHBEARER is supported but is not configured as the client's authentication mechanism,RD_KAFKA_RESP_ERR__INVALID_ARG
if no error string is supplied.RD_EXPORT rd_kafka_error_t* rd_kafka_init_transactions | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Initialize transactions for the producer instance.
This function ensures any transactions initiated by previous instances of the producer with the same transactional.id
are completed. If the previous instance failed with a transaction in progress the previous transaction will be aborted. This function needs to be called before any other transactional or produce functions are called when the transactional.id
is configured.
If the last transaction had begun completion (following transaction commit) but not yet finished, this function will await the previous transaction's completion.
When any previous transactions have been fenced this function will acquire the internal producer id and epoch, used in all future transactional messages issued by this producer instance.
rk | Producer instance. |
timeout_ms | The maximum time to block. On timeout the operation may continue in the background, depending on state, and it is okay to call init_transactions() again. If an infinite timeout (-1) is passed, the timeout will be adjusted to 2 * transaction.timeout.ms . |
timeout_ms
milliseconds.timeout_ms
(retriable), RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE if the transaction coordinator is not available (retriable), RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS if a previous transaction would not complete within timeout_ms
(retriable), RD_KAFKA_RESP_ERR__STATE if transactions have already been started or upon fatal error, RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE if the broker(s) do not support transactions (<Apache Kafka 0.11), this also raises a fatal error, RD_KAFKA_RESP_ERR_INVALID_TRANSACTION_TIMEOUT if the configured transaction.timeout.ms
is outside the broker-configured range, this also raises a fatal error, RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been configured for the producer instance, RD_KAFKA_RESP_ERR__INVALID_ARG if rk
is not a producer instance, or timeout_ms
is out of range. Other error codes not listed here may be returned, depending on broker version.RD_EXPORT rd_kafka_error_t* rd_kafka_begin_transaction | ( | rd_kafka_t * | rk | ) |
Begin a new transaction.
rd_kafka_init_transactions() must have been called successfully (once) before this function is called.
Upon successful return from this function the application has to perform at least one of the following operations within transaction.timeout.ms
to avoid timing out the transaction on the broker:
Any messages produced, offsets sent (rd_kafka_send_offsets_to_transaction()), etc, after the successful return of this function will be part of the transaction and committed or aborted atomatically.
Finish the transaction by calling rd_kafka_commit_transaction() or abort the transaction by calling rd_kafka_abort_transaction().
rk | Producer instance. |
rk
is not a producer instance. Other error codes not listed here may be returned, depending on broker version.RD_EXPORT rd_kafka_error_t* rd_kafka_send_offsets_to_transaction | ( | rd_kafka_t * | rk, |
const rd_kafka_topic_partition_list_t * | offsets, | ||
const rd_kafka_consumer_group_metadata_t * | cgmetadata, | ||
int | timeout_ms | ||
) |
Sends a list of topic partition offsets to the consumer group coordinator for cgmetadata
, and marks the offsets as part part of the current transaction. These offsets will be considered committed only if the transaction is committed successfully.
The offsets should be the next message your application will consume, i.e., the last processed message's offset + 1 for each partition. Either track the offsets manually during processing or use rd_kafka_position() (on the consumer) to get the current offsets for the partitions assigned to the consumer.
Use this method at the end of a consume-transform-produce loop prior to committing the transaction with rd_kafka_commit_transaction().
rk | Producer instance. |
offsets | List of offsets to commit to the consumer group upon successful commit of the transaction. Offsets should be the next message to consume, e.g., last processed message + 1. |
cgmetadata | The current consumer group metadata as returned by rd_kafka_consumer_group_metadata() on the consumer instance the provided offsets were consumed from. |
timeout_ms | Maximum time allowed to register the offsets on the broker. |
enable.auto.commit
to false on the consumer).offsets
will be ignored, if there are no valid offsets in offsets
the function will return NULL and no action will be taken.offsets
and cgmetadata
parameters. Failure to do so risks inconsistent state between what is actually included in the transaction and what the application thinks is included in the transaction.rk
is not a producer instance, or if the consumer_group_id
or offsets
are empty. Other error codes not listed here may be returned, depending on broker version.RD_EXPORT rd_kafka_error_t* rd_kafka_commit_transaction | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Commit the current transaction (as started with rd_kafka_begin_transaction()).
Any outstanding messages will be flushed (delivered) before actually committing the transaction.
If any of the outstanding messages fail permanently the current transaction will enter the abortable error state and this function will return an abortable error, in this case the application must call rd_kafka_abort_transaction() before attempting a new transaction with rd_kafka_begin_transaction().
rk | Producer instance. |
timeout_ms | The maximum time to block. On timeout the operation may continue in the background, depending on state, and it is okay to call this function again. Pass -1 to use the remaining transaction timeout, this is the recommended use. |
timeout_ms
. Using other values risk internal state desynchronization in case any of the underlying protocol requests fail.timeout_ms
expires, which ever comes first. On timeout the application may call the function again.timeout_ms
, this is a retriable error as the commit continues in the background, RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH if the current producer transaction has been fenced by a newer producer instance, RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED if the producer is no longer authorized to perform transactional operations, RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been configured for the producer instance, RD_KAFKA_RESP_ERR__INVALID_ARG if rk
is not a producer instance, Other error codes not listed here may be returned, depending on broker version.RD_EXPORT rd_kafka_error_t* rd_kafka_abort_transaction | ( | rd_kafka_t * | rk, |
int | timeout_ms | ||
) |
Aborts the ongoing transaction.
This function should also be used to recover from non-fatal abortable transaction errors. Any outstanding messages will be purged and fail with RD_KAFKA_RESP_ERR__PURGE_INFLIGHT or RD_KAFKA_RESP_ERR__PURGE_QUEUE. See rd_kafka_purge() for details.
rk | Producer instance. |
timeout_ms | The maximum time to block. On timeout the operation may continue in the background, depending on state, and it is okay to call this function again. Pass -1 to use the remaining transaction timeout, this is the recommended use. |
timeout_ms
. Using other values risk internal state desynchronization in case any of the underlying protocol requests fail.timeout_ms
expires, which ever comes first. On timeout the application may call the function again. If the application has enabled RD_KAFKA_EVENT_DR it must serve the event queue in a separate thread since rd_kafka_flush() will not serve delivery reports in this mode.timeout_ms
, this is a retriable error as the commit continues in the background, RD_KAFKA_RESP_ERR_INVALID_PRODUCER_EPOCH if the current producer transaction has been fenced by a newer producer instance, RD_KAFKA_RESP_ERR_TRANSACTIONAL_ID_AUTHORIZATION_FAILED if the producer is no longer authorized to perform transactional operations, RD_KAFKA_RESP_ERR__NOT_CONFIGURED if transactions have not been configured for the producer instance, RD_KAFKA_RESP_ERR__INVALID_ARG if rk
is not a producer instance, Other error codes not listed here may be returned, depending on broker version.