librdkafka
The Apache Kafka C/C++ client library
rdkafka.h File Reference

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_t
 Topic+Partition place holder. More...
 
struct  rd_kafka_topic_partition_list_t
 A growable list of Topic+Partitions. More...
 
struct  rd_kafka_message_t
 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_t
 Broker information. More...
 
struct  rd_kafka_metadata_partition_t
 Partition information. More...
 
struct  rd_kafka_metadata_topic_t
 Topic information. More...
 
struct  rd_kafka_metadata_t
 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...
 

Functions

Kafka messages
RD_EXPORT void rd_kafka_message_destroy (rd_kafka_message_t *rkmessage)
 Frees resources for rkmessage and hands ownership back to rdkafka.
 
static RD_INLINE const char *RD_UNUSED 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.
 
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...
 
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 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 *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(*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 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 *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 *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 *opaque, void *msg_opaque)
 Consistent-Random 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_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 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...
 
Simple Consumer API (legacy): Queue consumers

The following ..._queue() functions are analogue to the functions above but reads messages from the provided queue rkqu instead. rkqu must have been previously created with rd_kafka_queue_new() and the topic consumer must have been started with rd_kafka_consume_start_queue() utilising the the same queue.

RD_EXPORT rd_kafka_message_trd_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 *opaque), void *opaque)
 Consume multiple messages from queue with callback. More...
 
Simple Consumer API (legacy): Topic+partition offset store.

If auto.commit.enable is true the offset is stored automatically prior to returning of the message(s) in each of the rd_kafka_consume*() functions above.

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 for topic rkt partition partition. 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_trd_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 down the KafkaConsumer. 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. 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. 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 *opaque), void *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...
 
Metadata API
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.
 
Client group information
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...
 
RD_EXPORT void rd_kafka_group_list_destroy (const struct rd_kafka_group_list *grplist)
 Release list memory.
 
Miscellaneous APIs
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 produced 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.
 
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...
 
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...
 

librdkafka version

#define RD_KAFKA_VERSION   0x000905ff
 librdkafka version More...
 
RD_EXPORT int rd_kafka_version (void)
 Returns the librdkafka version as integer. More...
 
RD_EXPORT const char * rd_kafka_version_str (void)
 Returns the librdkafka version as string. More...
 

Constants, errors, types

#define RD_KAFKA_DEBUG_CONTEXTS   "all,generic,broker,topic,metadata,queue,msg,protocol,cgrp,security,fetch,feature"
 Supported debug contexts. (compile time) More...
 
enum  rd_kafka_type_t {
  RD_KAFKA_PRODUCER,
  RD_KAFKA_CONSUMER
}
 rd_kafka_t handle type. More...
 
enum  rd_kafka_timestamp_type_t {
  RD_KAFKA_TIMESTAMP_NOT_AVAILABLE,
  RD_KAFKA_TIMESTAMP_CREATE_TIME,
  RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME
}
 
enum  rd_kafka_resp_err_t {
  RD_KAFKA_RESP_ERR__BEGIN = -200,
  RD_KAFKA_RESP_ERR__BAD_MSG = -199,
  RD_KAFKA_RESP_ERR__BAD_COMPRESSION = -198,
  RD_KAFKA_RESP_ERR__DESTROY = -197,
  RD_KAFKA_RESP_ERR__FAIL = -196,
  RD_KAFKA_RESP_ERR__TRANSPORT = -195,
  RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE = -194,
  RD_KAFKA_RESP_ERR__RESOLVE = -193,
  RD_KAFKA_RESP_ERR__MSG_TIMED_OUT = -192,
  RD_KAFKA_RESP_ERR__PARTITION_EOF = -191,
  RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION = -190,
  RD_KAFKA_RESP_ERR__FS = -189,
  RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC = -188,
  RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN = -187,
  RD_KAFKA_RESP_ERR__INVALID_ARG = -186,
  RD_KAFKA_RESP_ERR__TIMED_OUT = -185,
  RD_KAFKA_RESP_ERR__QUEUE_FULL = -184,
  RD_KAFKA_RESP_ERR__ISR_INSUFF = -183,
  RD_KAFKA_RESP_ERR__NODE_UPDATE = -182,
  RD_KAFKA_RESP_ERR__SSL = -181,
  RD_KAFKA_RESP_ERR__WAIT_COORD = -180,
  RD_KAFKA_RESP_ERR__UNKNOWN_GROUP = -179,
  RD_KAFKA_RESP_ERR__IN_PROGRESS = -178,
  RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS = -177,
  RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION = -176,
  RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS = -175,
  RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS = -174,
  RD_KAFKA_RESP_ERR__CONFLICT = -173,
  RD_KAFKA_RESP_ERR__STATE = -172,
  RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL = -171,
  RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED = -170,
  RD_KAFKA_RESP_ERR__AUTHENTICATION = -169,
  RD_KAFKA_RESP_ERR__NO_OFFSET = -168,
  RD_KAFKA_RESP_ERR__OUTDATED = -167,
  RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE = -166,
  RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE = -165,
  RD_KAFKA_RESP_ERR__WAIT_CACHE = -164,
  RD_KAFKA_RESP_ERR__END = -100,
  RD_KAFKA_RESP_ERR_UNKNOWN = -1,
  RD_KAFKA_RESP_ERR_NO_ERROR = 0,
  RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE = 1,
  RD_KAFKA_RESP_ERR_INVALID_MSG = 2,
  RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART = 3,
  RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE = 4,
  RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE = 5,
  RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION = 6,
  RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT = 7,
  RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE = 8,
  RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE = 9,
  RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE = 10,
  RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH = 11,
  RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE = 12,
  RD_KAFKA_RESP_ERR_NETWORK_EXCEPTION = 13,
  RD_KAFKA_RESP_ERR_GROUP_LOAD_IN_PROGRESS = 14,
  RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE = 15,
  RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP = 16,
  RD_KAFKA_RESP_ERR_TOPIC_EXCEPTION = 17,
  RD_KAFKA_RESP_ERR_RECORD_LIST_TOO_LARGE = 18,
  RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS = 19,
  RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND = 20,
  RD_KAFKA_RESP_ERR_INVALID_REQUIRED_ACKS = 21,
  RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION = 22,
  RD_KAFKA_RESP_ERR_INCONSISTENT_GROUP_PROTOCOL = 23,
  RD_KAFKA_RESP_ERR_INVALID_GROUP_ID = 24,
  RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID = 25,
  RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT = 26,
  RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS = 27,
  RD_KAFKA_RESP_ERR_INVALID_COMMIT_OFFSET_SIZE = 28,
  RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED = 29,
  RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED = 30,
  RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED = 31,
  RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP = 32,
  RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM = 33,
  RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE = 34,
  RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION = 35,
  RD_KAFKA_RESP_ERR_TOPIC_ALREADY_EXISTS = 36,
  RD_KAFKA_RESP_ERR_INVALID_PARTITIONS = 37,
  RD_KAFKA_RESP_ERR_INVALID_REPLICATION_FACTOR = 38,
  RD_KAFKA_RESP_ERR_INVALID_REPLICA_ASSIGNMENT = 39,
  RD_KAFKA_RESP_ERR_INVALID_CONFIG = 40,
  RD_KAFKA_RESP_ERR_NOT_CONTROLLER = 41,
  RD_KAFKA_RESP_ERR_INVALID_REQUEST = 42,
  RD_KAFKA_RESP_ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT = 43,
  RD_KAFKA_RESP_ERR_END_ALL
}
 Error codes. More...
 
RD_EXPORT const char * rd_kafka_get_debug_contexts (void)
 Retrieve supported debug contexts for use with the "debug" configuration property. (runtime) More...
 
RD_EXPORT void rd_kafka_get_err_descs (const struct rd_kafka_err_desc **errdescs, size_t *cntp)
 Returns the full list of error codes.
 
RD_EXPORT const char * rd_kafka_err2str (rd_kafka_resp_err_t err)
 Returns a human readable representation of a kafka error. More...
 
RD_EXPORT const char * rd_kafka_err2name (rd_kafka_resp_err_t err)
 Returns the error code name (enum name). More...
 
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. More...
 
RD_EXPORT 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: More...
 
RD_EXPORT int rd_kafka_errno (void)
 Returns the thread-local system errno. More...
 
RD_EXPORT void rd_kafka_topic_partition_destroy (rd_kafka_topic_partition_t *rktpar)
 Destroy a rd_kafka_topic_partition_t. More...
 
RD_EXPORT rd_kafka_topic_partition_list_trd_kafka_topic_partition_list_new (int size)
 Create a new list/vector Topic+Partition container. More...
 
RD_EXPORT void rd_kafka_topic_partition_list_destroy (rd_kafka_topic_partition_list_t *rkparlist)
 Free all resources used by the list and the list itself.
 
RD_EXPORT rd_kafka_topic_partition_trd_kafka_topic_partition_list_add (rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition)
 Add topic+partition to list. More...
 
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. More...
 
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. More...
 
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. More...
 
RD_EXPORT rd_kafka_topic_partition_list_trd_kafka_topic_partition_list_copy (const rd_kafka_topic_partition_list_t *src)
 Make a copy of an existing list. More...
 
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. More...
 
RD_EXPORT rd_kafka_topic_partition_trd_kafka_topic_partition_list_find (rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition)
 Find element by topic and partition. More...
 
RD_EXPORT void rd_kafka_topic_partition_list_sort (rd_kafka_topic_partition_list_t *rktparlist, int(*cmp)(const void *a, const void *b, void *opaque), void *opaque)
 Sort list using comparator cmp. 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(opaque)
 
#define RD_KAFKA_V_MSGFLAGS(msgflags)
 
#define RD_KAFKA_V_TIMESTAMP(timestamp)
 
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
}
 Var-arg tag types. More...
 

Configuration interface

Main/global configuration property 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...
 
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.
 
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_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()
 
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_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_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_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)
 
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...
 
RD_EXPORT const 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...
 
RD_EXPORT const 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.
 

Main Kafka and Topic object handles

#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 const char * rd_kafka_name (const rd_kafka_t *rk)
 Returns Kafka handle name.
 
RD_EXPORT char * rd_kafka_memberid (const rd_kafka_t *rk)
 Returns this client's broker-assigned group member id. 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...
 
RD_EXPORT const 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.
 
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_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_message_trd_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 *opaque), void *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
 
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 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...
 

Event interface

The event API provides an alternative pollable non-callback interface to librdkafka's message and event queues.

#define RD_KAFKA_EVENT_NONE   0x0
 
#define RD_KAFKA_EVENT_DR   0x1
 
#define RD_KAFKA_EVENT_FETCH   0x2
 
#define RD_KAFKA_EVENT_LOG   0x4
 
#define RD_KAFKA_EVENT_ERROR   0x8
 
#define RD_KAFKA_EVENT_REBALANCE   0x10
 
#define RD_KAFKA_EVENT_OFFSET_COMMIT   0x20
 
typedef int rd_kafka_event_type_t
 Event types.
 
typedef struct rd_kafka_op_s rd_kafka_event_t
 
RD_EXPORT rd_kafka_event_type_t rd_kafka_event_type (const rd_kafka_event_t *rkev)
 
RD_EXPORT const char * rd_kafka_event_name (const rd_kafka_event_t *rkev)
 
RD_EXPORT void rd_kafka_event_destroy (rd_kafka_event_t *rkev)
 Destroy an event. More...
 
RD_EXPORT const rd_kafka_message_trd_kafka_event_message_next (rd_kafka_event_t *rkev)
 
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. More...
 
RD_EXPORT size_t rd_kafka_event_message_count (rd_kafka_event_t *rkev)
 
RD_EXPORT rd_kafka_resp_err_t rd_kafka_event_error (rd_kafka_event_t *rkev)
 
RD_EXPORT const char * rd_kafka_event_error_string (rd_kafka_event_t *rkev)
 
RD_EXPORT void * rd_kafka_event_opaque (rd_kafka_event_t *rkev)
 
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. More...
 
RD_EXPORT rd_kafka_topic_partition_list_trd_kafka_event_topic_partition_list (rd_kafka_event_t *rkev)
 
RD_EXPORT rd_kafka_topic_partition_trd_kafka_event_topic_partition (rd_kafka_event_t *rkev)
 
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. More...
 
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. More...
 

Detailed Description

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.

See also
For the C++ interface see rdkafkacpp.h

Macro Definition Documentation

§ RD_KAFKA_VERSION

#define RD_KAFKA_VERSION   0x000905ff

librdkafka version

Interpreted as hex MM.mm.rr.xx:

  • MM = Major
  • mm = minor
  • rr = revision
  • xx = pre-release id (0xff is the final release)

E.g.: 0x000801ff = 0.8.1

Remarks
This value should only be used during compile time, for runtime checks of version use rd_kafka_version()

§ RD_KAFKA_DEBUG_CONTEXTS

#define RD_KAFKA_DEBUG_CONTEXTS   "all,generic,broker,topic,metadata,queue,msg,protocol,cgrp,security,fetch,feature"

Supported debug contexts. (compile time)

Deprecated:
This compile time value may be outdated at runtime due to linking another version of the library. Use rd_kafka_get_debug_contexts() instead.

§ RD_KAFKA_V_END

#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

§ RD_KAFKA_V_TOPIC

#define RD_KAFKA_V_TOPIC (   topic)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_TOPIC, const char *, topic), \
(const char *)topic
Definition: rdkafka.h:730

Topic name (const char *)

§ RD_KAFKA_V_RKT

#define RD_KAFKA_V_RKT (   rkt)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_RKT, rd_kafka_topic_t *, rkt), \
(rd_kafka_topic_t *)rkt
Definition: rdkafka.h:731

Topic object (rd_kafka_topic_t *)

§ RD_KAFKA_V_PARTITION

#define RD_KAFKA_V_PARTITION (   partition)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_PARTITION, int32_t, partition), \
(int32_t)partition
Definition: rdkafka.h:732

Partition (int32_t)

§ RD_KAFKA_V_VALUE

#define RD_KAFKA_V_VALUE (   VALUE,
  LEN 
)
Value:
_LRK_TYPECHECK2(RD_KAFKA_VTYPE_VALUE, void *, VALUE, size_t, LEN), \
(void *)VALUE, (size_t)LEN
Definition: rdkafka.h:733

Message value/payload pointer and length (void *, size_t)

§ RD_KAFKA_V_KEY

#define RD_KAFKA_V_KEY (   KEY,
  LEN 
)
Value:
_LRK_TYPECHECK2(RD_KAFKA_VTYPE_KEY, const void *, KEY, size_t, LEN), \
(void *)KEY, (size_t)LEN
Definition: rdkafka.h:734

Message key pointer and length (const void *, size_t)

§ RD_KAFKA_V_OPAQUE

#define RD_KAFKA_V_OPAQUE (   opaque)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_OPAQUE, void *, opaque), \
(void *)opaque
Definition: rdkafka.h:735

Opaque pointer (void *)

§ RD_KAFKA_V_MSGFLAGS

#define RD_KAFKA_V_MSGFLAGS (   msgflags)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_MSGFLAGS, int, msgflags), \
(int)msgflags
Definition: rdkafka.h:736

Message flags (int)

See also
RD_KAFKA_MSG_F_COPY, et.al.

§ RD_KAFKA_V_TIMESTAMP

#define RD_KAFKA_V_TIMESTAMP (   timestamp)
Value:
_LRK_TYPECHECK(RD_KAFKA_VTYPE_TIMESTAMP, int64_t, timestamp), \
(int64_t)timestamp
Definition: rdkafka.h:737

Timestamp (int64_t)

§ RD_KAFKA_PARTITION_UA

#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.

§ RD_KAFKA_OFFSET_BEGINNING

#define RD_KAFKA_OFFSET_BEGINNING   -2

Start consuming from beginning of kafka partition queue: oldest msg

§ RD_KAFKA_OFFSET_END

#define RD_KAFKA_OFFSET_END   -1

Start consuming from end of kafka partition queue: next msg

§ RD_KAFKA_OFFSET_STORED

#define RD_KAFKA_OFFSET_STORED   -1000

Start consuming from offset retrieved from offset store

§ RD_KAFKA_OFFSET_INVALID

#define RD_KAFKA_OFFSET_INVALID   -1001

Invalid offset

§ RD_KAFKA_OFFSET_TAIL

#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.

§ RD_KAFKA_MSG_F_FREE

#define RD_KAFKA_MSG_F_FREE   0x1

Producer message flags.

Delegate freeing of payload to rdkafka.

§ RD_KAFKA_MSG_F_COPY

#define RD_KAFKA_MSG_F_COPY   0x2

rdkafka will make a copy of the payload.

§ RD_KAFKA_MSG_F_BLOCK

#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 indefinately blocking on the produce() call when the message queue is full.

§ RD_KAFKA_EVENT_DR

#define RD_KAFKA_EVENT_DR   0x1

Producer Delivery report batch

§ RD_KAFKA_EVENT_FETCH

#define RD_KAFKA_EVENT_FETCH   0x2

Fetched message (consumer)

§ RD_KAFKA_EVENT_LOG

#define RD_KAFKA_EVENT_LOG   0x4

Log message

§ RD_KAFKA_EVENT_ERROR

#define RD_KAFKA_EVENT_ERROR   0x8

Error

§ RD_KAFKA_EVENT_REBALANCE

#define RD_KAFKA_EVENT_REBALANCE   0x10

Group rebalance (consumer)

§ RD_KAFKA_EVENT_OFFSET_COMMIT

#define RD_KAFKA_EVENT_OFFSET_COMMIT   0x20

Offset commit result

Enumeration Type Documentation

§ rd_kafka_type_t

rd_kafka_t handle type.

See also
rd_kafka_new()
Enumerator
RD_KAFKA_PRODUCER 

Producer client

RD_KAFKA_CONSUMER 

Consumer client

§ rd_kafka_timestamp_type_t

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

§ 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..>".

See also
Use rd_kafka_err2str() to translate an error code a human readable 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.

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__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_NOT_LEADER_FOR_PARTITION 

Not leader for partition

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_GROUP_LOAD_IN_PROGRESS 

Group coordinator load in progress

RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE 

Group coordinator not available

RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP 

Not coordinator for group

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_vtype_t

Var-arg tag types.

See also
rd_kafka_producev()
Enumerator
RD_KAFKA_VTYPE_END 

va-arg sentinel

RD_KAFKA_VTYPE_TOPIC 

(const char *) Topic name

RD_KAFKA_VTYPE_RKT 

(rd_kafka_topic_t *) Topic handle

RD_KAFKA_VTYPE_PARTITION 

(int32_t) Partition

RD_KAFKA_VTYPE_VALUE 

(void *, size_t) Message value (payload)

RD_KAFKA_VTYPE_KEY 

(void *, size_t) Message key

RD_KAFKA_VTYPE_OPAQUE 

(void *) Application opaque

RD_KAFKA_VTYPE_MSGFLAGS 

(int) RD_KAFKA_MSG_F_.. flags

RD_KAFKA_VTYPE_TIMESTAMP 

(int64_t) Milliseconds since epoch UTC

§ rd_kafka_conf_res_t

Configuration result type.

Enumerator
RD_KAFKA_CONF_UNKNOWN 

Unknown configuration name.

RD_KAFKA_CONF_INVALID 

Invalid configuration value.

RD_KAFKA_CONF_OK 

Configuration okay

Function Documentation

§ rd_kafka_version()

RD_EXPORT int rd_kafka_version ( void  )

Returns the librdkafka version as integer.

Returns
Version integer.
See also
See RD_KAFKA_VERSION for how to parse the integer format.
Use rd_kafka_version_str() to retreive the version as a string.

§ rd_kafka_version_str()

RD_EXPORT const char* rd_kafka_version_str ( void  )

Returns the librdkafka version as string.

Returns
Version string

§ rd_kafka_get_debug_contexts()

RD_EXPORT const char* rd_kafka_get_debug_contexts ( void  )

Retrieve supported debug contexts for use with the "debug" configuration property. (runtime)

Returns
Comma-separated list of available debugging contexts.

§ rd_kafka_err2str()

RD_EXPORT const char* rd_kafka_err2str ( rd_kafka_resp_err_t  err)

Returns a human readable representation of a kafka error.

Parameters
errError code to translate

§ rd_kafka_err2name()

RD_EXPORT const char* rd_kafka_err2name ( rd_kafka_resp_err_t  err)

Returns the error code name (enum name).

Parameters
errError code to translate

§ rd_kafka_last_error()

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.

Remarks
The last error is stored per-thread, if multiple rd_kafka_t handles are used in the same application thread the developer needs to make sure rd_kafka_last_error() is called immediately after a failed API call.

§ rd_kafka_errno2err()

RD_EXPORT 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:

Parameters
errnoxSystem errno value to convert
Returns
Appropriate error code for errnox
Remarks
A better alternative is to call rd_kafka_last_error() immediately after any of the above functions return -1 or NULL.
See also
rd_kafka_last_error()

§ rd_kafka_errno()

RD_EXPORT 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 expsing the errno librdkafka uses.

Remarks
The value is local to the current calling thread.

§ rd_kafka_topic_partition_destroy()

RD_EXPORT void rd_kafka_topic_partition_destroy ( rd_kafka_topic_partition_t rktpar)

Destroy a rd_kafka_topic_partition_t.

Remarks
This must not be called for elements in a topic partition list.

§ rd_kafka_topic_partition_list_new()

RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_topic_partition_list_new ( int  size)

Create a new list/vector Topic+Partition container.

Parameters
sizeInitial allocated size used when the expected number of elements is known or can be estimated. Avoids reallocation and possibly relocation of the elems array.
Returns
A newly allocated Topic+Partition list.
Remarks
Use rd_kafka_topic_partition_list_destroy() to free all resources in use by a list and the list itself.
See also
rd_kafka_topic_partition_list_add()

§ rd_kafka_topic_partition_list_add()

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.

Parameters
rktparlistList to extend
topicTopic name (copied)
partitionPartition id
Returns
The object which can be used to fill in additionals fields.

§ rd_kafka_topic_partition_list_add_range()

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.

Parameters
rktparlistList to extend
topicTopic name (copied)
startStart partition of range
stopLast partition of range (inclusive)

§ rd_kafka_topic_partition_list_del()

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.

Parameters
rktparlistList to modify
topicTopic name to match
partitionPartition to match
Returns
1 if partition was found (and removed), else 0.
Remarks
Any held indices to elems[] are unusable after this call returns 1.

§ rd_kafka_topic_partition_list_del_by_idx()

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.

Returns
1 if partition was found (and removed), else 0.
See also
rd_kafka_topic_partition_list_del()

§ rd_kafka_topic_partition_list_copy()

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.

Parameters
srcThe existing list to copy.
Returns
A new list fully populated to be identical to src

§ rd_kafka_topic_partition_list_set_offset()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION if partition was not found in the list.

§ rd_kafka_topic_partition_list_find()

RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_topic_partition_list_find ( rd_kafka_topic_partition_list_t rktparlist,
const char *  topic,
int32_t  partition 
)

Find element by topic and partition.

Returns
a pointer to the first matching element, or NULL if not found.

§ rd_kafka_topic_partition_list_sort()

RD_EXPORT void rd_kafka_topic_partition_list_sort ( rd_kafka_topic_partition_list_t rktparlist,
int(*)(const void *a, const void *b, void *opaque)  cmp,
void *  opaque 
)

Sort list using comparator cmp.

If cmp is NULL the default comparator will be used that sorts by ascending topic name and partition.

§ rd_kafka_message_timestamp()

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.

Returns
message timestamp, or -1 if not available.
Remarks
Message timestamps require broker version 0.10.0 or later.

§ rd_kafka_conf_new()

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.:

rd_kafka_conf_t *myconf;
myconf = rd_kafka_conf_new();
res = rd_kafka_conf_set(myconf, "socket.timeout.ms", "600",
errstr, sizeof(errstr));
if (res != RD_KAFKA_CONF_OK)
die("%s\n", errstr);
rk = rd_kafka_new(..., myconf);

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.

Returns
A new rd_kafka_conf_t object with defaults set.
See also
rd_kafka_conf_set(), rd_kafka_conf_destroy()

§ rd_kafka_conf_set()

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 sub-sequent rd_kafka_conf_set_default_topic_conf() calls will replace the current default topic configuration.

Returns
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_kafka_conf_set_dr_cb()

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_kafka_conf_set_dr_msg_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, or the retry counter for temporary errors has been exhausted.

An application must call rd_kafka_poll() at regular intervals to serve queued delivery report callbacks.

§ rd_kafka_conf_set_rebalance_cb()

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.

Remarks
In this latter case (arbitrary error), the application must call rd_kafka_assign(rk, NULL) to synchronize state.

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 assinging/revocation, such as fetching offsets from an alternate location (on assign) or manually committing offsets (on revoke).

Remarks
The partitions list is destroyed by librdkafka on return return from the rebalance_cb and must not be freed or saved by the application.

The following example shows the application's responsibilities:

static void rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
void *opaque) {
switch (err)
{
// application may load offets from arbitrary external
// storage here and update \p partitions
rd_kafka_assign(rk, partitions);
break;
if (manual_commits) // Optional explicit manual commit
rd_kafka_commit(rk, partitions, 0); // sync commit
rd_kafka_assign(rk, NULL);
break;
default:
handle_unlikely_error(err);
rd_kafka_assign(rk, NULL); // sync state
break;
}
}

§ rd_kafka_conf_set_offset_commit_cb()

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 error

§ rd_kafka_conf_set_error_cb()

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 critical errors back to the application.

If no error_cb is registered then the errors will be logged instead.

§ rd_kafka_conf_set_throttle_cb()

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.

Remarks
Requires broker version 0.9.0 or later.

§ rd_kafka_conf_set_log_cb()

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()

Remarks
The log_cb will be called spontaneously from librdkafka's internal threads unless logs have been forwarded to a poll queue through 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_kafka_conf_set_stats_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 handle
  • json - String containing the statistics data in JSON format
  • json_len - Length of json string.
  • opaque - Application-provided opaque.

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.

§ rd_kafka_conf_set_socket_cb()

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.

Default:

  • on linux: racefree CLOEXEC
  • others : non-racefree CLOEXEC
Remarks
The callback will be called from an internal librdkafka thread.

§ rd_kafka_conf_set_connect_cb()

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.

Remarks
The callback will be called from an internal librdkafka thread.

§ rd_kafka_conf_set_closesocket_cb()

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()).

Remarks
The callback will be called from an internal librdkafka thread.

§ rd_kafka_conf_set_open_cb()

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:

  • on linux: racefree CLOEXEC
  • others : non-racefree CLOEXEC
Remarks
The callback will be called from an internal librdkafka thread.

§ rd_kafka_conf_set_default_topic_conf()

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_kafka_conf_get()

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.

Returns
RD_KAFKA_CONF_OK if the property name matched, else RD_KAFKA_CONF_UNKNOWN.

§ rd_kafka_topic_conf_get()

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.

See also
rd_kafka_conf_get()

§ rd_kafka_conf_dump()

RD_EXPORT const 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().

§ rd_kafka_topic_conf_dump()

RD_EXPORT const 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_kafka_topic_conf_new()

RD_EXPORT rd_kafka_topic_conf_t* rd_kafka_topic_conf_new ( void  )

Create topic configuration object.

See also
Same semantics as for rd_kafka_conf_new().

§ rd_kafka_topic_conf_set()

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().

Returns
rd_kafka_conf_res_t to indicate success or failure.

§ rd_kafka_topic_conf_set_partitioner_cb()

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.

Partitioner function constraints:

  • MUST NOT call any rd_kafka_*() functions except: rd_kafka_topic_partition_available()
  • MUST NOT block or execute for prolonged periods of time.
  • MUST return a value between 0 and partition_cnt-1, or the special RD_KAFKA_PARTITION_UA value if partitioning could not be performed.

§ rd_kafka_topic_partition_available()

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).

Returns
1 if the partition is available, else 0.
Warning
This function must only be called from inside a partitioner function

§ rd_kafka_msg_partitioner_random()

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 *  opaque,
void *  msg_opaque 
)

Random partitioner.

Will try not to return unavailable partitions.

Returns
a random partition between 0 and partition_cnt - 1.

§ rd_kafka_msg_partitioner_consistent()

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 *  opaque,
void *  msg_opaque 
)

Consistent partitioner.

Uses consistent hashing to map identical keys onto identical partitions.

Returns
a "random" partition between 0 and partition_cnt - 1 based on the CRC value of the key

§ rd_kafka_msg_partitioner_consistent_random()

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 *  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.

Returns
a "random" partition between 0 and partition_cnt - 1 based on the CRC value of the key (if provided)

§ rd_kafka_new()

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 sub-sequently. 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.

Remarks
RD_KAFKA_CONSUMER: When a new 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.
An application must only use one of these groups of APIs on a given rd_kafka_t RD_KAFKA_CONSUMER handle.
Returns
The Kafka handle on success or NULL on error (see errstr)
See also
To destroy the Kafka handle, use rd_kafka_destroy().

§ rd_kafka_destroy()

RD_EXPORT void rd_kafka_destroy ( rd_kafka_t *  rk)

Destroy Kafka handle.

Remarks
This is a blocking operation.

§ rd_kafka_memberid()

RD_EXPORT char* rd_kafka_memberid ( const rd_kafka_t *  rk)

Returns this client's broker-assigned group member id.

Remarks
This currently requires the high-level KafkaConsumer
Returns
An allocated string containing the current broker-assigned group member id, or NULL if not available. The application must free the string with free() or rd_kafka_mem_free()

§ rd_kafka_topic_new()

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 sub-sequently. 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.

Returns
the new topic handle or NULL on error (use rd_kafka_errno2err() to convert system errno to an rd_kafka_resp_err_t error code.
See also
rd_kafka_topic_destroy()

§ rd_kafka_topic_destroy()

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().

Remarks
Since topic objects are refcounted (both internally and for the app) the topic object might not actually be destroyed by this call, but the application must consider the object destroyed.

§ rd_kafka_poll()

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 indefinately for an event, provide -1.

Remarks
An application should make sure to call poll() at regular intervals to serve any queued callbacks waiting to be called.

Events:

Returns
the number of events served.

§ rd_kafka_yield()

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.

Remarks
This function MUST ONLY be called from within a librdkafka callback.

§ rd_kafka_pause_partitions()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR

§ rd_kafka_resume_partitions()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR

§ rd_kafka_query_watermark_offsets()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on failure.

§ rd_kafka_get_watermark_offsets()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on failure.
Remarks
Shall only be used with an active consumer instance.

§ rd_kafka_offsets_for_times()

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.

Remarks
Duplicate Topic+Partitions are not supported.
Per-partition errors may be returned in rd_kafka_topic_partition_t.err
Returns
an error code for general errors, else RD_KAFKA_RESP_ERR_NO_ERROR in which case per-partition errors might be set.

§ rd_kafka_mem_free()

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.

Remarks
rd_kafka_mem_free() must only be used for pointers returned by APIs that explicitly mention using this function for freeing.

§ rd_kafka_queue_new()

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_kafka_queue_destroy()

RD_EXPORT void rd_kafka_queue_destroy ( rd_kafka_queue_t *  rkqu)

Destroy a queue, purging all of its enqueued messages.

§ rd_kafka_queue_get_main()

RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_main ( rd_kafka_t *  rk)
Returns
a reference to the main librdkafka event queue. This is the queue served by rd_kafka_poll().

Use rd_kafka_queue_destroy() to loose the reference.

§ rd_kafka_queue_get_consumer()

RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_consumer ( rd_kafka_t *  rk)
Returns
a reference to the librdkafka consumer queue. This is the queue served by rd_kafka_consumer_poll().

Use rd_kafka_queue_destroy() to loose the reference.

Remarks
rd_kafka_queue_destroy() MUST be called on this queue prior to calling rd_kafka_consumer_close().

§ rd_kafka_queue_get_partition()

RD_EXPORT rd_kafka_queue_t* rd_kafka_queue_get_partition ( rd_kafka_t *  rk,
const char *  topic,
int32_t  partition 
)
Returns
a reference to the partition's queue, or NULL if partition is invalid.

Use rd_kafka_queue_destroy() to loose the reference.

Remarks
rd_kafka_queue_destroy() MUST be called on this queue
This function only works on consumers.

§ rd_kafka_queue_forward()

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.

Remarks
Regardless of whether dst is NULL or not, after calling this function, src will not forward it's fetch queue to the consumer queue.

§ rd_kafka_set_log_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.

Parameters
rkquQueue to forward logs to. If the value is NULL the logs are forwarded to the main queue.
Remarks
The configuration property log.queue MUST also be set to true.
librdkafka maintains its own reference to the provided queue.
Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error.

§ rd_kafka_queue_length()

RD_EXPORT size_t rd_kafka_queue_length ( rd_kafka_queue_t *  rkqu)
Returns
the current number of elements in queue.

§ rd_kafka_queue_io_event_enable()

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.

Remarks
When using forwarded queues the IO event must only be enabled on the final forwarded-to (destination) queue.

§ rd_kafka_consume_start()

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:

  • RD_KAFKA_OFFSET_BEGINNING
  • RD_KAFKA_OFFSET_END
  • RD_KAFKA_OFFSET_STORED
  • RD_KAFKA_OFFSET_TAIL

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().

Returns
0 on success or -1 on error in which case errno is set accordingly:
  • EBUSY - Conflicts with an existing or previous subscription (RD_KAFKA_RESP_ERR__CONFLICT)
  • EINVAL - Invalid offset, or incomplete configuration (lacking group.id) (RD_KAFKA_RESP_ERR__INVALID_ARG)
  • ESRCH - requested partition is invalid. (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)
  • ENOENT - topic is unknown in the Kafka cluster. (RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)

Use rd_kafka_errno2err() to convert sytem errno to rd_kafka_resp_err_t

§ rd_kafka_consume_start_queue()

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_kafka_consume_stop()

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.

Returns
0 on success or -1 on error (see errno).

§ rd_kafka_seek()

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 not 0 the call will wait this long for the seek to be performed. If the timeout is reached the internal state will be unknown 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 triggers a fetch queue barrier flush.

Returns
RD_KAFKA_RESP_ERR__NO_ERROR on success else an error code.

§ rd_kafka_consume()

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().

Returns a message object on success or 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):

  • ETIMEDOUT - timeout_ms was reached with no new messages fetched.
  • ENOENT - 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_kafka_consume_batch()

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.

Returns
the number of rkmessages added in rkmessages, or -1 on error (same error codes as for rd_kafka_consume().
See also
rd_kafka_consume()

§ rd_kafka_consume_callback()

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 *opaque)  consume_cb,
void *  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 opaque argument is passed to the 'consume_cb' as opaque.

Returns
the number of messages processed or -1 on error.
See also
rd_kafka_consume()

§ rd_kafka_consume_queue()

RD_EXPORT rd_kafka_message_t* rd_kafka_consume_queue ( rd_kafka_queue_t *  rkqu,
int  timeout_ms 
)

Consume from queue.

See also
rd_kafka_consume()

§ rd_kafka_consume_batch_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.

See also
rd_kafka_consume_batch()

§ rd_kafka_consume_callback_queue()

RD_EXPORT int rd_kafka_consume_callback_queue ( rd_kafka_queue_t *  rkqu,
int  timeout_ms,
void(*)(rd_kafka_message_t *rkmessage, void *opaque)  consume_cb,
void *  opaque 
)

Consume multiple messages from queue with callback.

See also
rd_kafka_consume_callback()

§ rd_kafka_offset_store()

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 for topic rkt partition partition.

The offset will be committed (written) to the offset store according to auto.commit.interval.ms.

Remarks
auto.commit.enable must be set to "false" when using this API.
Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error.

§ rd_kafka_subscribe()

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 by the librdkafka assignor: 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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success or RD_KAFKA_RESP_ERR__INVALID_ARG if list is empty, contains invalid topics or regexes.

§ rd_kafka_subscription()

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.

Returns
An error code on failure, otherwise topic is updated to point to a newly allocated topic list (possibly empty).
Remarks
The application is responsible for calling rd_kafka_topic_partition_list_destroy on the returned list.

§ rd_kafka_consumer_poll()

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.

Remarks
An application should make sure to call consumer_poll() at regular intervals, even if no messages are expected, to serve any queued callbacks waiting to be called. This is especially important when a rebalance_cb has been registered as it needs to be called and handled properly to synchronize internal consumer state.
Returns
A message object which is a proper message if ->err is RD_KAFKA_RESP_ERR_NO_ERROR, or an event or error for any other value.
See also
rd_kafka_message_t

§ rd_kafka_consumer_close()

RD_EXPORT rd_kafka_resp_err_t rd_kafka_consumer_close ( rd_kafka_t *  rk)

Close down the KafkaConsumer.

Remarks
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. The maximum blocking time is roughly limited to session.timeout.ms.
Returns
An error code indicating if the consumer close was succesful or not.
Remarks
The application still needs to call rd_kafka_destroy() after this call finishes to clean up the underlying handle resources.

§ rd_kafka_assign()

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.

When used from a rebalance callback the application shall pass the partition list passed to the callback (or a copy of it) (even if the list is empty) rather than NULL to maintain internal join state.

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.

§ rd_kafka_assignment()

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.

Returns
An error code on failure, otherwise partitions is updated to point to a newly allocated partition list (possibly empty).
Remarks
The application is responsible for calling rd_kafka_topic_partition_list_destroy on the returned list.

§ rd_kafka_commit()

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. 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.

§ rd_kafka_commit_message()

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.

See also
rd_kafka_commit

§ rd_kafka_commit_queue()

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 *opaque)  cb,
void *  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. opaque is passed to the callback.

If using the event API the callback is ignored and the offset commit result will be returned as an RD_KAFKA_EVENT_COMMIT event. The 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.

See also
rd_kafka_commit()
rd_kafka_conf_set_offset_commit_cb()

§ rd_kafka_committed()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the 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_kafka_position()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the 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_kafka_produce()

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 asynch non-blocking API.

partition is the target partition, either:

  • RD_KAFKA_PARTITION_UA (unassigned) for automatic partitioning using the topic's partitioner function, or
  • a fixed partition (0..N)

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.

.._F_FREE and .._F_COPY are mutually exclusive.

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 delivery report callback (dr_cb) for referencing this message.

Returns 0 on success or -1 on error in which case errno is set accordingly:

  • ENOBUFS - maximum number of outstanding messages has been reached: "queue.buffering.max.messages" (RD_KAFKA_RESP_ERR__QUEUE_FULL)
  • EMSGSIZE - message is larger than configured max size: "messages.max.bytes". (RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE)
  • ESRCH - requested partition is unknown in the Kafka cluster. (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)
  • ENOENT - topic is unknown in the Kafka cluster. (RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)
See also
Use rd_kafka_errno2err() to convert errno to rdkafka error code.

§ rd_kafka_producev()

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.

Returns
RD_KAFKA_RESP_ERR_NO_ERROR on success, else an error code.
See also
rd_kafka_produce, RD_KAFKA_V_END

§ rd_kafka_produce_batch()

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:

  • payload,len Message payload and length
  • key,key_len Optional message key
  • _private Message opaque pointer (msg_opaque)
  • err Will be set according to success or failure. Application only needs to check for errors if return value != message_cnt.
Returns
the number of messages succesfully enqueued for producing.

§ rd_kafka_flush()

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.

Remarks
This function will call rd_kafka_poll() and thus trigger callbacks.
Returns
RD_KAFKA_RESP_ERR__TIMED_OUT if timeout_ms was reached before all outstanding requests were completed, else RD_KAFKA_RESP_ERR_NO_ERROR

§ rd_kafka_metadata()

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 topic
  • metadatap pointer to hold metadata result. The *metadatap pointer must be released with rd_kafka_metadata_destroy().
  • timeout_ms maximum response time before failing.

Returns RD_KAFKA_RESP_ERR_NO_ERROR on success (in which case *metadatap) will be set, else RD_KAFKA_RESP_ERR__TIMED_OUT on timeout or other error code on error.

§ rd_kafka_list_groups()

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.

Returns
RD_KAFKA_RESP_ERR__NO_ERROR on success and grplistp is updated to point to a newly allocated list of groups. Else returns an error code on failure and grplistp remains untouched.
See also
Use rd_kafka_group_list_destroy() to release list memory.

§ rd_kafka_brokers_add()

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"

Returns
the number of brokers successfully added.
Remarks
Brokers may also be defined with the metadata.broker.list or bootstrap.servers configuration property (preferred method).

§ rd_kafka_set_logger()

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.

Deprecated:
Use rd_kafka_conf_set_log_cb()
Remarks
rk may be passed as NULL in the callback.

§ rd_kafka_set_log_level()

RD_EXPORT void rd_kafka_set_log_level ( rd_kafka_t *  rk,
int  level 
)

Specifies the maximum logging level produced by internal kafka logging and debugging.

If the "debug" configuration property is set the level is automatically adjusted to LOG_DEBUG (7).

§ rd_kafka_outq_len()

RD_EXPORT int rd_kafka_outq_len ( rd_kafka_t *  rk)

Returns the current out queue length.

The out queue contains messages waiting to be sent to, or acknowledged by, the broker.

An application should wait for this queue to reach zero before terminating to make sure outstanding requests (such as offset commits) are fully processed.

Returns
number of messages in the out queue.

§ rd_kafka_dump()

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_kafka_thread_cnt()

RD_EXPORT int rd_kafka_thread_cnt ( void  )

Retrieve the current number of threads in use by librdkafka.

Used by regression tests.

§ rd_kafka_wait_destroyed()

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. Since rd_kafka_destroy() is an asynch operation the rd_kafka_wait_destroyed() function can be used for applications where a clean shutdown is required.

§ rd_kafka_poll_set_consumer()

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()).

Warning
It is not permitted to call rd_kafka_poll() after directing the main queue with rd_kafka_poll_set_consumer().

§ rd_kafka_event_type()

RD_EXPORT rd_kafka_event_type_t rd_kafka_event_type ( const rd_kafka_event_t *  rkev)
Returns
the event type for the given event.
Remarks
As a convenience it is okay to pass rkev as NULL in which case RD_KAFKA_EVENT_NONE is returned.

§ rd_kafka_event_name()

RD_EXPORT const char* rd_kafka_event_name ( const rd_kafka_event_t *  rkev)
Returns
the event type's name for the given event.
Remarks
As a convenience it is okay to pass rkev as NULL in which case the name for RD_KAFKA_EVENT_NONE is returned.

§ rd_kafka_event_destroy()

RD_EXPORT void rd_kafka_event_destroy ( rd_kafka_event_t *  rkev)

Destroy an event.

Remarks
Any references to this event, such as extracted messages, will not be usable after this call.
As a convenience it is okay to pass rkev as NULL in which case no action is performed.

§ rd_kafka_event_message_next()

RD_EXPORT const rd_kafka_message_t* rd_kafka_event_message_next ( rd_kafka_event_t *  rkev)
Returns
the next message from an event.

Call repeatedly until it returns NULL.

Event types:

  • RD_KAFKA_EVENT_FETCH (1 message)
  • RD_KAFKA_EVENT_DR (>=1 message(s))
Remarks
The returned message(s) MUST NOT be freed with rd_kafka_message_destroy().

§ rd_kafka_event_message_array()

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_KAFKA_EVENT_FETCH (1 message)
  • RD_KAFKA_EVENT_DR (>=1 message(s))
Returns
the number of messages extracted.

§ rd_kafka_event_message_count()

RD_EXPORT size_t rd_kafka_event_message_count ( rd_kafka_event_t *  rkev)
Returns
the number of remaining messages in the event.

Event types:

  • RD_KAFKA_EVENT_FETCH (1 message)
  • RD_KAFKA_EVENT_DR (>=1 message(s))

§ rd_kafka_event_error()

RD_EXPORT rd_kafka_resp_err_t rd_kafka_event_error ( rd_kafka_event_t *  rkev)
Returns
the error code for the event.

Event types:

  • all

§ rd_kafka_event_error_string()

RD_EXPORT const char* rd_kafka_event_error_string ( rd_kafka_event_t *  rkev)
Returns
the error string (if any). An application should check that rd_kafka_event_error() returns non-zero before calling this function.

Event types:

  • all

§ rd_kafka_event_opaque()

RD_EXPORT void* rd_kafka_event_opaque ( rd_kafka_event_t *  rkev)
Returns
the user opaque (if any)

Event types:

  • RD_KAFKA_OFFSET_COMMIT

§ rd_kafka_event_log()

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_KAFKA_EVENT_LOG
Returns
0 on success or -1 if unsupported event type.

§ rd_kafka_event_topic_partition_list()

RD_EXPORT rd_kafka_topic_partition_list_t* rd_kafka_event_topic_partition_list ( rd_kafka_event_t *  rkev)
Returns
the topic partition list from the event.
Remarks
The list MUST NOT be freed with rd_kafka_topic_partition_list_destroy()

Event types:

  • RD_KAFKA_EVENT_REBALANCE
  • RD_KAFKA_EVENT_OFFSET_COMMIT

§ rd_kafka_event_topic_partition()

RD_EXPORT rd_kafka_topic_partition_t* rd_kafka_event_topic_partition ( rd_kafka_event_t *  rkev)
Returns
a newly allocated topic_partition container, if applicable for the event type, else NULL.
Remarks
The returned pointer MUST be freed with rd_kafka_topic_partition_destroy().

Event types: RD_KAFKA_EVENT_ERROR (for partition level errors)

§ rd_kafka_queue_poll()

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.

Returns
an event, or NULL.
Remarks
Use rd_kafka_event_destroy() to free the event.

§ rd_kafka_queue_poll_callback()

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.

Returns
the number of events served.
Remarks
This API must only be used for queues with callbacks registered for all expected event types. E.g., not a message queue.