@@ -58,6 +58,7 @@ extern "C" {
58
58
#include <basetsd.h>
59
59
typedef SSIZE_T ssize_t ;
60
60
#define RD_UNUSED
61
+ #define RD_INLINE __inline
61
62
#define RD_DEPRECATED
62
63
#undef RD_EXPORT
63
64
#ifdef LIBRDKAFKA_EXPORTS
@@ -68,6 +69,7 @@ typedef SSIZE_T ssize_t;
68
69
69
70
#else
70
71
#define RD_UNUSED __attribute__((unused))
72
+ #define RD_INLINE inline
71
73
#define RD_EXPORT
72
74
#define RD_DEPRECATED __attribute__((deprecated))
73
75
#endif
@@ -96,7 +98,7 @@ typedef SSIZE_T ssize_t;
96
98
* @remark This value should only be used during compile time,
97
99
* for runtime checks of version use rd_kafka_version()
98
100
*/
99
- #define RD_KAFKA_VERSION 0x00090100
101
+ #define RD_KAFKA_VERSION 0x000901ff
100
102
101
103
/**
102
104
* @brief Returns the librdkafka version as integer.
@@ -141,6 +143,19 @@ typedef enum rd_kafka_type_t {
141
143
} rd_kafka_type_t ;
142
144
143
145
146
+ /**
147
+ * @enum Timestamp types
148
+ *
149
+ * @sa rd_kafka_message_timestamp()
150
+ */
151
+ typedef enum rd_kafka_timestamp_type_t {
152
+ RD_KAFKA_TIMESTAMP_NOT_AVAILABLE , /**< Timestamp not available */
153
+ RD_KAFKA_TIMESTAMP_CREATE_TIME , /**< Message creation time */
154
+ RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME /**< Log append time */
155
+ } rd_kafka_timestamp_type_t ;
156
+
157
+
158
+
144
159
/**
145
160
* @brief Retrieve supported debug contexts for use with the \c \"debug\"
146
161
* configuration property. (runtime)
@@ -252,6 +267,8 @@ typedef enum {
252
267
RD_KAFKA_RESP_ERR__AUTHENTICATION = -169 ,
253
268
/** No stored offset */
254
269
RD_KAFKA_RESP_ERR__NO_OFFSET = -168 ,
270
+ /** Outdated */
271
+ RD_KAFKA_RESP_ERR__OUTDATED = -167 ,
255
272
/** End internal error codes */
256
273
RD_KAFKA_RESP_ERR__END = -100 ,
257
274
@@ -322,6 +339,14 @@ typedef enum {
322
339
RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED = 30 ,
323
340
/** Cluster authorization failed */
324
341
RD_KAFKA_RESP_ERR_CLUSTER_AUTHORIZATION_FAILED = 31 ,
342
+ /** Invalid timestamp */
343
+ RD_KAFKA_RESP_ERR_INVALID_TIMESTAMP = 32 ,
344
+ /** Unsupported SASL mechanism */
345
+ RD_KAFKA_RESP_ERR_UNSUPPORTED_SASL_MECHANISM = 33 ,
346
+ /** Illegal SASL state */
347
+ RD_KAFKA_RESP_ERR_ILLEGAL_SASL_STATE = 34 ,
348
+ /** Unuspported version */
349
+ RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION = 35 ,
325
350
326
351
RD_KAFKA_RESP_ERR_END_ALL ,
327
352
} rd_kafka_resp_err_t ;
@@ -530,6 +555,37 @@ rd_kafka_topic_partition_list_add_range (rd_kafka_topic_partition_list_t
530
555
531
556
532
557
558
+ /**
559
+ * @brief Delete partition from list.
560
+ *
561
+ * @param rktparlist List to modify
562
+ * @param topic Topic name to match
563
+ * @param partition Partition to match
564
+ *
565
+ * @returns 1 if partition was found (and removed), else 0.
566
+ *
567
+ * @remark Any held indices to elems[] are unusable after this call returns 1.
568
+ */
569
+ RD_EXPORT
570
+ int
571
+ rd_kafka_topic_partition_list_del (rd_kafka_topic_partition_list_t * rktparlist ,
572
+ const char * topic , int32_t partition );
573
+
574
+
575
+ /**
576
+ * @brief Delete partition from list by elems[] index.
577
+ *
578
+ * @returns 1 if partition was found (and removed), else 0.
579
+ *
580
+ * @sa rd_kafka_topic_partition_list_del()
581
+ */
582
+ RD_EXPORT
583
+ int
584
+ rd_kafka_topic_partition_list_del_by_idx (
585
+ rd_kafka_topic_partition_list_t * rktparlist ,
586
+ int idx );
587
+
588
+
533
589
/**
534
590
* @brief Make a copy of an existing list.
535
591
*
@@ -585,9 +641,9 @@ rd_kafka_topic_partition_list_find (rd_kafka_topic_partition_list_t *rktparlist,
585
641
586
642
/**
587
643
* @brief A Kafka message as returned by the \c rd_kafka_consume*() family
588
- * of functions.
644
+ * of functions as well as provided to the Producer \c dr_msg_cb() .
589
645
*
590
- * This object has two purposes:
646
+ * For the consumer this object has two purposes:
591
647
* - provide the application with a consumed message. (\c err == 0)
592
648
* - report per-topic+partition consumer errors (\c err != 0)
593
649
*
@@ -600,7 +656,8 @@ typedef struct rd_kafka_message_s {
600
656
rd_kafka_resp_err_t err ; /**< Non-zero for error signaling. */
601
657
rd_kafka_topic_t * rkt ; /**< Topic */
602
658
int32_t partition ; /**< Partition */
603
- void * payload ; /**< Depends on the value of \c err :
659
+ void * payload ; /**< Producer: original message payload.
660
+ * Consumer: Depends on the value of \c err :
604
661
* - \c err==0: Message payload.
605
662
* - \c err!=0: Error string */
606
663
size_t len ; /**< Depends on the value of \c err :
@@ -640,7 +697,7 @@ void rd_kafka_message_destroy(rd_kafka_message_t *rkmessage);
640
697
* @brief Returns the error string for an errored rd_kafka_message_t or NULL if
641
698
* there was no error.
642
699
*/
643
- static __inline const char *
700
+ static RD_INLINE const char *
644
701
RD_UNUSED
645
702
rd_kafka_message_errstr (const rd_kafka_message_t * rkmessage ) {
646
703
if (!rkmessage -> err )
@@ -652,6 +709,25 @@ rd_kafka_message_errstr(const rd_kafka_message_t *rkmessage) {
652
709
return rd_kafka_err2str (rkmessage -> err );
653
710
}
654
711
712
+
713
+
714
+ /**
715
+ * @brief Returns the message timestamp for a consumed message.
716
+ *
717
+ * The timestamp is the number of milliseconds since the epoch (UTC).
718
+ *
719
+ * \p tstype is updated to indicate the type of timestamp.
720
+ *
721
+ * @returns message timestamp, or -1 if not available.
722
+ *
723
+ * @remark Message timestamps require broker version 0.10.0 or later.
724
+ */
725
+ RD_EXPORT
726
+ int64_t rd_kafka_message_timestamp (const rd_kafka_message_t * rkmessage ,
727
+ rd_kafka_timestamp_type_t * tstype );
728
+
729
+
730
+
655
731
/**@}*/
656
732
657
733
@@ -805,7 +881,11 @@ void rd_kafka_conf_set_consume_cb (rd_kafka_conf_t *conf,
805
881
* such as fetching offsets from an alternate location (on assign)
806
882
* or manually committing offsets (on revoke).
807
883
*
808
- * The following example show's the application's responsibilities:
884
+ * @remark The \p partitions list is destroyed by librdkafka on return
885
+ * return from the rebalance_cb and must not be freed or
886
+ * saved by the application.
887
+ *
888
+ * The following example shows the application's responsibilities:
809
889
* @code
810
890
* static void rebalance_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
811
891
* rd_kafka_topic_partition_list_t *partitions,
@@ -885,15 +965,16 @@ void rd_kafka_conf_set_error_cb(rd_kafka_conf_t *conf,
885
965
/**
886
966
* @brief Set throttle callback.
887
967
*
888
- * The throttle callback is used in conjunction with
889
- * \c quota.support.enable=true to forward broker throttle times to the
968
+ * The throttle callback is used to forward broker throttle times to the
890
969
* application for Produce and Fetch (consume) requests.
891
970
*
892
971
* Callbacks are triggered whenever a non-zero throttle time is returned by
893
972
* the broker, or when the throttle time drops back to zero.
894
973
*
895
974
* An application must call rd_kafka_poll() or rd_kafka_consumer_poll() at
896
975
* regular intervals to serve queued callbacks.
976
+ *
977
+ * @remark Requires broker version 0.9.0 or later.
897
978
*/
898
979
RD_EXPORT
899
980
void rd_kafka_conf_set_throttle_cb (rd_kafka_conf_t * conf ,
@@ -1604,7 +1685,7 @@ int rd_kafka_consume_start_queue(rd_kafka_topic_t *rkt, int32_t partition,
1604
1685
* all messages currently in the local queue.
1605
1686
*
1606
1687
* NOTE: To enforce synchronisation this call will block until the internal
1607
- * fetcher has terminated and offsets are commited to configured
1688
+ * fetcher has terminated and offsets are committed to configured
1608
1689
* storage method.
1609
1690
*
1610
1691
* The application needs to be stop all consumers before calling
@@ -1787,7 +1868,7 @@ int rd_kafka_consume_callback_queue(rd_kafka_queue_t *rkqu,
1787
1868
/**
1788
1869
* @brief Store offset \p offset for topic \p rkt partition \p partition.
1789
1870
*
1790
- * The offset will be commited (written) to the offset store according
1871
+ * The offset will be committed (written) to the offset store according
1791
1872
* to \c `auto.commit.interval.ms`.
1792
1873
*
1793
1874
* @remark \c `auto.commit.enable` must be set to "false" when using this API.
@@ -1873,6 +1954,7 @@ rd_kafka_message_t *rd_kafka_consumer_poll (rd_kafka_t *rk, int timeout_ms);
1873
1954
* @remark This call will block until the consumer has revoked its assignment,
1874
1955
* calling the \c rebalance_cb if it is configured, committed offsets
1875
1956
* to broker, and left the consumer group.
1957
+ * The maximum blocking time is roughly limited to session.timeout.ms.
1876
1958
*
1877
1959
* @returns An error code indicating if the consumer close was succesful
1878
1960
* or not.
@@ -1939,7 +2021,30 @@ rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
1939
2021
1940
2022
1941
2023
/**
1942
- * @brief Retrieve committed positions (offsets) for topics+partitions.
2024
+ * @brief Retrieve committed offsets for topics+partitions.
2025
+ *
2026
+ * The \p offset field of each requested partition will either be set to
2027
+ * stored offset or to RD_KAFKA_OFFSET_INVALID in case there was no stored
2028
+ * offset for that partition.
2029
+ *
2030
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the
2031
+ * \p offset or \p err field of each \p partitions' element is filled
2032
+ * in with the stored offset, or a partition specific error.
2033
+ * Else returns an error code.
2034
+ */
2035
+ RD_EXPORT rd_kafka_resp_err_t
2036
+ rd_kafka_committed (rd_kafka_t * rk ,
2037
+ rd_kafka_topic_partition_list_t * partitions ,
2038
+ int timeout_ms );
2039
+
2040
+
2041
+
2042
+ /**
2043
+ * @brief Retrieve current positions (offsets) for topics+partitions.
2044
+ *
2045
+ * The \p offset field of each requested partition will be set to the offset
2046
+ * of the last consumed message + 1, or RD_KAFKA_OFFSET_INVALID in case there was
2047
+ * no previous message.
1943
2048
*
1944
2049
* @returns RD_KAFKA_RESP_ERR_NO_ERROR on success in which case the
1945
2050
* \p offset or \p err field of each \p partitions' element is filled
@@ -1948,8 +2053,8 @@ rd_kafka_commit_message (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
1948
2053
*/
1949
2054
RD_EXPORT rd_kafka_resp_err_t
1950
2055
rd_kafka_position (rd_kafka_t * rk ,
1951
- rd_kafka_topic_partition_list_t * partitions ,
1952
- int timeout_ms );
2056
+ rd_kafka_topic_partition_list_t * partitions );
2057
+
1953
2058
1954
2059
/**@}*/
1955
2060
0 commit comments