From b75c4d37338f1ae7bd8c8843e45e5aeac348d514 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 30 Nov 2022 23:19:14 +0100 Subject: [PATCH 01/40] Add partition leader_epoch and change rktpar _private to point to glue object The on-demand glue object contains the previous rktp reference and a new leader_epoch field. This is for KIP-320 --- CHANGELOG.md | 1 + src-cpp/HandleImpl.cpp | 7 ++- src-cpp/rdkafkacpp.h | 6 ++ src-cpp/rdkafkacpp_int.h | 24 ++++++-- src/rdkafka.h | 33 ++++++++++- src/rdkafka_assignment.c | 12 +++- src/rdkafka_cgrp.c | 4 +- src/rdkafka_partition.c | 120 ++++++++++++++++++++++++++------------- src/rdkafka_partition.h | 76 +++++++++++++++++++++++-- src/rdkafka_request.c | 12 ++-- 10 files changed, 228 insertions(+), 67 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b189c7bb90..c3e1889291 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -75,6 +75,7 @@ configuration property (refer [examples/rdkafka_complex_consumer_example.c] dependencies for its bundled librdkafka builds, as everything but cyrus-sasl is now built-in. There are bundled builds with and without linking to cyrus-sasl for maximum compatibility. + * Added `rd_kafka_topic_partition_get_leader_epoch()` (and `set..()`). ## Fixes diff --git a/src-cpp/HandleImpl.cpp b/src-cpp/HandleImpl.cpp index 9e295a979a..7aa2f2939b 100644 --- a/src-cpp/HandleImpl.cpp +++ b/src-cpp/HandleImpl.cpp @@ -391,6 +391,8 @@ rd_kafka_topic_partition_list_t *partitions_to_c_parts( rd_kafka_topic_partition_t *rktpar = rd_kafka_topic_partition_list_add( c_parts, tpi->topic_.c_str(), tpi->partition_); rktpar->offset = tpi->offset_; + if (tpi->leader_epoch_ != -1) + rd_kafka_topic_partition_set_leader_epoch(rktpar, tpi->leader_epoch_); } return c_parts; @@ -412,8 +414,9 @@ void update_partitions_from_c_parts( dynamic_cast(partitions[j]); if (!strcmp(p->topic, pp->topic_.c_str()) && p->partition == pp->partition_) { - pp->offset_ = p->offset; - pp->err_ = static_cast(p->err); + pp->offset_ = p->offset; + pp->err_ = static_cast(p->err); + pp->leader_epoch_ = rd_kafka_topic_partition_get_leader_epoch(p); } } } diff --git a/src-cpp/rdkafkacpp.h b/src-cpp/rdkafkacpp.h index 4c1014cfd2..7db203ee55 100644 --- a/src-cpp/rdkafkacpp.h +++ b/src-cpp/rdkafkacpp.h @@ -1978,6 +1978,12 @@ class RD_EXPORT TopicPartition { /** @returns error code (if applicable) */ virtual ErrorCode err() const = 0; + + /** @brief Get partition leader epoch, or -1 if not known or relevant. */ + virtual int32_t get_leader_epoch() = 0; + + /* @brief Set partition leader epoch. */ + virtual void set_leader_epoch(int32_t leader_epoch) = 0; }; diff --git a/src-cpp/rdkafkacpp_int.h b/src-cpp/rdkafkacpp_int.h index 5f7cefd642..0cb23aa009 100644 --- a/src-cpp/rdkafkacpp_int.h +++ b/src-cpp/rdkafkacpp_int.h @@ -1227,21 +1227,24 @@ class TopicPartitionImpl : public TopicPartition { topic_(topic), partition_(partition), offset_(RdKafka::Topic::OFFSET_INVALID), - err_(ERR_NO_ERROR) { + err_(ERR_NO_ERROR), + leader_epoch_(-1) { } TopicPartitionImpl(const std::string &topic, int partition, int64_t offset) : topic_(topic), partition_(partition), offset_(offset), - err_(ERR_NO_ERROR) { + err_(ERR_NO_ERROR), + leader_epoch_(-1) { } TopicPartitionImpl(const rd_kafka_topic_partition_t *c_part) { - topic_ = std::string(c_part->topic); - partition_ = c_part->partition; - offset_ = c_part->offset; - err_ = static_cast(c_part->err); + topic_ = std::string(c_part->topic); + partition_ = c_part->partition; + offset_ = c_part->offset; + err_ = static_cast(c_part->err); + leader_epoch_ = rd_kafka_topic_partition_get_leader_epoch(c_part); // FIXME: metadata } @@ -1266,6 +1269,14 @@ class TopicPartitionImpl : public TopicPartition { offset_ = offset; } + int32_t get_leader_epoch() { + return leader_epoch_; + } + + void set_leader_epoch(int32_t leader_epoch) { + leader_epoch_ = leader_epoch_; + } + std::ostream &operator<<(std::ostream &ostrm) const { return ostrm << topic_ << " [" << partition_ << "]"; } @@ -1274,6 +1285,7 @@ class TopicPartitionImpl : public TopicPartition { int partition_; int64_t offset_; ErrorCode err_; + int32_t leader_epoch_; }; diff --git a/src/rdkafka.h b/src/rdkafka.h index 3569f873b8..b9c1decefe 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -898,7 +898,9 @@ typedef struct rd_kafka_topic_partition_s { void *opaque; /**< Opaque value for application use */ rd_kafka_resp_err_t err; /**< Error code, depending on use. */ void *_private; /**< INTERNAL USE ONLY, - * INITIALIZE TO ZERO, DO NOT TOUCH */ + * INITIALIZE TO ZERO, DO NOT TOUCH, + * DO NOT COPY, DO NOT SHARE WITH OTHER + * rd_kafka_t INSTANCES. */ } rd_kafka_topic_partition_t; @@ -910,6 +912,32 @@ RD_EXPORT void rd_kafka_topic_partition_destroy(rd_kafka_topic_partition_t *rktpar); +/** + * @brief Sets the partition's leader epoch (use -1 to clear). + * + * @param rktpar Partition object. + * @param leader_epoch Partition leader epoch, use -1 to reset. + * + * @remark See KIP-320 for more information. + */ +RD_EXPORT +void rd_kafka_topic_partition_set_leader_epoch( + rd_kafka_topic_partition_t *rktpar, + int32_t leader_epoch); + +/** + * @returns the partition's leader epoch, if relevant and known, + * else -1. + * + * @param rktpar Partition object. + * + * @remark See KIP-320 for more information. + */ +RD_EXPORT +int32_t rd_kafka_topic_partition_get_leader_epoch( + const rd_kafka_topic_partition_t *rktpar); + + /** * @brief A growable list of Topic+Partitions. * @@ -1429,7 +1457,8 @@ typedef struct rd_kafka_message_s { * for retried messages when * idempotence is enabled. */ void *_private; /**< Consumer: - * - rdkafka private pointer: DO NOT MODIFY + * - rdkafka private pointer: + * DO NOT MODIFY, DO NOT COPY. * Producer: * - dr_msg_cb: * msg_opaque from produce() call or diff --git a/src/rdkafka_assignment.c b/src/rdkafka_assignment.c index 5f05683d94..ea0d533af9 100644 --- a/src/rdkafka_assignment.c +++ b/src/rdkafka_assignment.c @@ -135,7 +135,9 @@ rd_kafka_assignment_apply_offsets(rd_kafka_t *rk, rd_kafka_topic_partition_t *rktpar; RD_KAFKA_TPLIST_FOREACH(rktpar, offsets) { - rd_kafka_toppar_t *rktp = rktpar->_private; /* May be NULL */ + /* May be NULL, borrow ref. */ + rd_kafka_toppar_t *rktp = + rd_kafka_topic_partition_toppar(rk, rktpar); if (!rd_kafka_topic_partition_list_del( rk->rk_consumer.assignment.queried, rktpar->topic, @@ -302,7 +304,9 @@ static int rd_kafka_assignment_serve_removals(rd_kafka_t *rk) { int valid_offsets = 0; RD_KAFKA_TPLIST_FOREACH(rktpar, rk->rk_consumer.assignment.removed) { - rd_kafka_toppar_t *rktp = rktpar->_private; /* Borrow ref */ + rd_kafka_toppar_t *rktp = + rd_kafka_topic_partition_ensure_toppar( + rk, rktpar, rd_true); /* Borrow ref */ int was_pending, was_queried; /* Remove partition from pending and querying lists, @@ -422,7 +426,9 @@ static int rd_kafka_assignment_serve_pending(rd_kafka_t *rk) { for (i = rk->rk_consumer.assignment.pending->cnt - 1; i >= 0; i--) { rd_kafka_topic_partition_t *rktpar = &rk->rk_consumer.assignment.pending->elems[i]; - rd_kafka_toppar_t *rktp = rktpar->_private; /* Borrow ref */ + /* Borrow ref */ + rd_kafka_toppar_t *rktp = + rd_kafka_topic_partition_ensure_toppar(rk, rktpar, rd_true); rd_assert(!rktp->rktp_started); diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 4934130ac3..1e5e10c610 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -5290,9 +5290,7 @@ rd_kafka_cgrp_owned_but_not_exist_partitions(rd_kafka_cgrp_t *rkcg) { result = rd_kafka_topic_partition_list_new( rkcg->rkcg_group_assignment->cnt); - rd_kafka_topic_partition_list_add0( - __FUNCTION__, __LINE__, result, curr->topic, - curr->partition, curr->_private); + rd_kafka_topic_partition_list_add_copy(result, curr); } return result; diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 352eb033be..d61386412a 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -2024,14 +2024,16 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, rko->rko_u.offset_fetch.partitions; int64_t offset = RD_KAFKA_OFFSET_INVALID; - rktp = offsets->elems[0]._private; + rktp = rd_kafka_topic_partition_get_toppar( + rk, &offsets->elems[0], rd_true /*create-on-miss*/); + if (!rko->rko_err) { /* Request succeeded but per-partition might have failed */ rko->rko_err = offsets->elems[0].err; offset = offsets->elems[0].offset; } - offsets->elems[0]._private = NULL; + rd_kafka_topic_partition_list_destroy(offsets); rko->rko_u.offset_fetch.partitions = NULL; @@ -2067,6 +2069,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, "offsets from brokers: %s", rd_kafka_err2str(rko->rko_err)); + /* Refcount from get_toppar() */ rd_kafka_toppar_destroy(rktp); break; @@ -2092,6 +2095,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, "available"); rd_kafka_toppar_unlock(rktp); + /* Refcount from get_toppar() */ rd_kafka_toppar_destroy(rktp); } break; @@ -2561,7 +2565,15 @@ rd_kafka_topic_partition_new_from_rktp(rd_kafka_toppar_t *rktp) { return rktpar; } - +/** + * @brief Destroy a partition private glue object. + */ +static void rd_kafka_topic_partition_private_destroy( + rd_kafka_topic_partition_private_t *parpriv) { + if (parpriv->rktp) + rd_kafka_toppar_destroy(parpriv->rktp); + rd_free(parpriv); +} static void rd_kafka_topic_partition_destroy0(rd_kafka_topic_partition_t *rktpar, @@ -2571,13 +2583,36 @@ rd_kafka_topic_partition_destroy0(rd_kafka_topic_partition_t *rktpar, if (rktpar->metadata) rd_free(rktpar->metadata); if (rktpar->_private) - rd_kafka_toppar_destroy((rd_kafka_toppar_t *)rktpar->_private); + rd_kafka_topic_partition_private_destroy( + (rd_kafka_topic_partition_private_t *)rktpar->_private); if (do_free) rd_free(rktpar); } +int32_t rd_kafka_topic_partition_get_leader_epoch( + const rd_kafka_topic_partition_t *rktpar) { + const rd_kafka_topic_partition_private_t *parpriv; + + if (!(parpriv = rktpar->_private)) + return -1; + + return parpriv->leader_epoch; +} + +void rd_kafka_topic_partition_set_leader_epoch( + rd_kafka_topic_partition_t *rktpar, + int32_t leader_epoch) { + rd_kafka_topic_partition_private_t *parpriv; + + parpriv = rd_kafka_topic_partition_get_private(rktpar); + + parpriv->leader_epoch = leader_epoch; +} + + + /** * @brief Destroy all partitions in list. * @@ -2632,12 +2667,13 @@ void rd_kafka_topic_partition_list_destroy_free(void *ptr) { /** - * Add a partition to an rktpar list. + * @brief Add a partition to an rktpar list. * The list must have enough room to fit it. * - * '_private' must be NULL or a valid 'rd_kafka_toppar_t *'. + * @param rktp Optional partition object that will be stored on the + * ._private object (with refcount increased). * - * Returns a pointer to the added element. + * @returns a pointer to the added element. */ rd_kafka_topic_partition_t * rd_kafka_topic_partition_list_add0(const char *func, @@ -2645,7 +2681,7 @@ rd_kafka_topic_partition_list_add0(const char *func, rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition, - rd_kafka_toppar_t *_private) { + rd_kafka_toppar_t *rktp) { rd_kafka_topic_partition_t *rktpar; if (rktparlist->cnt == rktparlist->size) rd_kafka_topic_partition_list_grow(rktparlist, 1); @@ -2656,9 +2692,10 @@ rd_kafka_topic_partition_list_add0(const char *func, rktpar->topic = rd_strdup(topic); rktpar->partition = partition; rktpar->offset = RD_KAFKA_OFFSET_INVALID; - rktpar->_private = _private; - if (_private) - rd_kafka_toppar_keep_fl(func, line, _private); + + if (rktp) + rd_kafka_topic_partition_get_private(rktpar)->rktp = + rd_kafka_toppar_keep_fl(func, line, rktp); return rktpar; } @@ -2706,6 +2743,9 @@ rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_upsert( */ void rd_kafka_topic_partition_update(rd_kafka_topic_partition_t *dst, const rd_kafka_topic_partition_t *src) { + const rd_kafka_topic_partition_private_t *srcpriv; + rd_kafka_topic_partition_private_t *dstpriv; + rd_dassert(!strcmp(dst->topic, src->topic)); rd_dassert(dst->partition == src->partition); rd_dassert(dst != src); @@ -2720,6 +2760,20 @@ void rd_kafka_topic_partition_update(rd_kafka_topic_partition_t *dst, ; memcpy(dst->metadata, src->metadata, dst->metadata_size); } + + if ((srcpriv = src->_private)) { + dstpriv = rd_kafka_topic_partition_get_private(dst); + if (srcpriv->rktp && !dstpriv->rktp) + dstpriv->rktp = rd_kafka_toppar_keep(srcpriv->rktp); + + rd_assert(dstpriv->rktp == srcpriv->rktp); + + dstpriv->leader_epoch = srcpriv->leader_epoch; + + } else if ((dstpriv = dst->_private)) { + /* No private object in source, reset the leader epoch. */ + dstpriv->leader_epoch = -1; + } } /** @@ -2730,9 +2784,9 @@ void rd_kafka_topic_partition_list_add_copy( const rd_kafka_topic_partition_t *rktpar) { rd_kafka_topic_partition_t *dst; - dst = rd_kafka_topic_partition_list_add0( - __FUNCTION__, __LINE__, rktparlist, rktpar->topic, - rktpar->partition, rktpar->_private); + dst = rd_kafka_topic_partition_list_add0(__FUNCTION__, __LINE__, + rktparlist, rktpar->topic, + rktpar->partition, NULL); rd_kafka_topic_partition_update(dst, rktpar); } @@ -2829,30 +2883,16 @@ rd_kafka_toppar_t * rd_kafka_topic_partition_ensure_toppar(rd_kafka_t *rk, rd_kafka_topic_partition_t *rktpar, rd_bool_t create_on_miss) { - if (!rktpar->_private) - rktpar->_private = rd_kafka_toppar_get2( - rk, rktpar->topic, rktpar->partition, 0, create_on_miss); - return rktpar->_private; -} + rd_kafka_topic_partition_private_t *parpriv; + parpriv = rd_kafka_topic_partition_get_private(rktpar); -/** - * @returns (and sets if necessary) the \p rktpar's _private / toppar. - * @remark a new reference is returned. - */ -rd_kafka_toppar_t * -rd_kafka_topic_partition_get_toppar(rd_kafka_t *rk, - rd_kafka_topic_partition_t *rktpar, - rd_bool_t create_on_miss) { - rd_kafka_toppar_t *rktp; + if (!parpriv->rktp) + parpriv->rktp = rd_kafka_toppar_get2( + rk, rktpar->topic, rktpar->partition, + 0 /* not ua on miss */, create_on_miss); - rktp = - rd_kafka_topic_partition_ensure_toppar(rk, rktpar, create_on_miss); - - if (rktp) - rd_kafka_toppar_keep(rktp); - - return rktp; + return parpriv->rktp; } @@ -3094,7 +3134,9 @@ int rd_kafka_topic_partition_list_set_offsets( *preamble = '\0'; /* Avoid warning */ if (from_rktp) { - rd_kafka_toppar_t *rktp = rktpar->_private; + rd_kafka_toppar_t *rktp = + rd_kafka_topic_partition_ensure_toppar(rk, rktpar, + rd_true); rd_kafka_toppar_lock(rktp); if (rk->rk_conf.debug & @@ -3175,10 +3217,8 @@ void rd_kafka_topic_partition_list_update_toppars( for (i = 0; i < rktparlist->cnt; i++) { rd_kafka_topic_partition_t *rktpar = &rktparlist->elems[i]; - if (!rktpar->_private) - rktpar->_private = rd_kafka_toppar_get2( - rk, rktpar->topic, rktpar->partition, - 0 /*not ua-on-miss*/, create_on_miss); + rd_kafka_topic_partition_ensure_toppar(rk, rktpar, + create_on_miss); } } diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index e869820ef8..38c6e7f633 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -584,7 +584,7 @@ rd_kafka_topic_partition_list_add0(const char *func, rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition, - rd_kafka_toppar_t *_private); + rd_kafka_toppar_t *rktp); rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_upsert( rd_kafka_topic_partition_list_t *rktparlist, @@ -658,15 +658,81 @@ int rd_kafka_topic_partition_list_cmp(const void *_a, const void *_b, int (*cmp)(const void *, const void *)); + +/** + * @struct This is a separately allocated glue object used in + * rd_kafka_topic_partition_t._private to allow referencing both + * an rktp and/or a leader epoch. Both are optional. + * The rktp, if non-NULL, owns a refcount. + * + * This glue object is not always set in ._private, but allocated on demand + * as necessary. + */ +typedef struct rd_kafka_topic_partition_private_s { + /** Reference to a toppar. Optional, may be NULL. */ + rd_kafka_toppar_t *rktp; + /** Leader epoch if known, else -1. */ + int32_t leader_epoch; +} rd_kafka_topic_partition_private_t; + + + +/** + * @returns (and creates if necessary) the ._private glue object. + */ +static RD_UNUSED RD_INLINE rd_kafka_topic_partition_private_t * +rd_kafka_topic_partition_get_private(rd_kafka_topic_partition_t *rktpar) { + rd_kafka_topic_partition_private_t *parpriv; + + if (!(parpriv = rktpar->_private)) { + parpriv = rd_calloc(1, sizeof(*parpriv)); + parpriv->leader_epoch = -1; + rktpar->_private = parpriv; + } + + return parpriv; +} + + +/** + * @returns the partition's rktp if set (no refcnt increase), else NULL. + */ +static RD_INLINE RD_UNUSED rd_kafka_toppar_t * +rd_kafka_topic_partition_toppar(rd_kafka_t *rk, + const rd_kafka_topic_partition_t *rktpar) { + const rd_kafka_topic_partition_private_t *parpriv; + + if ((parpriv = rktpar->_private)) + return parpriv->rktp; + + return NULL; +} + rd_kafka_toppar_t * rd_kafka_topic_partition_ensure_toppar(rd_kafka_t *rk, rd_kafka_topic_partition_t *rktpar, rd_bool_t create_on_miss); -rd_kafka_toppar_t *rd_kafka_topic_partition_get_toppar( - rd_kafka_t *rk, - rd_kafka_topic_partition_t *rktpar, - rd_bool_t create_on_miss) RD_WARN_UNUSED_RESULT; +/** + * @returns (and sets if necessary) the \p rktpar's ._private. + * @remark a new reference is returned. + */ +static RD_INLINE RD_UNUSED rd_kafka_toppar_t * +rd_kafka_topic_partition_get_toppar(rd_kafka_t *rk, + rd_kafka_topic_partition_t *rktpar, + rd_bool_t create_on_miss) { + rd_kafka_toppar_t *rktp; + + rktp = + rd_kafka_topic_partition_ensure_toppar(rk, rktpar, create_on_miss); + + if (rktp) + rd_kafka_toppar_keep(rktp); + + return rktp; +} + + void rd_kafka_topic_partition_list_update_toppars( rd_kafka_t *rk, diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index d837ea31e6..4240affe39 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -780,12 +780,8 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk, seen_cnt++; - if (!(rktp = rktpar->_private)) { - rktp = rd_kafka_toppar_get2( - rkb->rkb_rk, topic_name, partition, 0, 0); - /* May be NULL if topic is not locally known */ - rktpar->_private = rktp; - } + rktp = rd_kafka_topic_partition_get_toppar( + rk, rktpar, rd_false /*no create on miss*/); /* broker reports invalid offset as -1 */ if (offset == -1) @@ -826,6 +822,10 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk, rktpar->metadata_size = RD_KAFKAP_STR_LEN(&metadata); } + + /* Loose ref from get_toppar() */ + if (rktp) + rd_kafka_toppar_destroy(rktp); } rd_kafka_buf_skip_tags(rkbuf); From 3ac9783348ba9b31d2344b637bedb83f0217a4db Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Thu, 1 Dec 2022 20:43:48 +0100 Subject: [PATCH 02/40] Update Kafka protocol enums --- src/rdkafka.c | 40 ++++++++++++++++++++++++++-------------- src/rdkafka_proto.h | 22 +++++++++++++++------- src/rdkafka_protocol.h | 11 ++++++++++- 3 files changed, 51 insertions(+), 22 deletions(-) diff --git a/src/rdkafka.c b/src/rdkafka.c index f5353e41ca..0d72a012b2 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -1569,8 +1569,6 @@ static void rd_kafka_stats_emit_broker_reqs(struct _stats_emit *st, [RD_KAFKAP_AlterReplicaLogDirs] = rd_true, [RD_KAFKAP_DescribeLogDirs] = rd_true, - [RD_KAFKAP_SaslAuthenticate] = rd_false, - [RD_KAFKAP_CreateDelegationToken] = rd_true, [RD_KAFKAP_RenewDelegationToken] = rd_true, [RD_KAFKAP_ExpireDelegationToken] = rd_true, @@ -1587,21 +1585,35 @@ static void rd_kafka_stats_emit_broker_reqs(struct _stats_emit *st, [RD_KAFKAP_AlterIsr] = rd_true, [RD_KAFKAP_UpdateFeatures] = rd_true, [RD_KAFKAP_Envelope] = rd_true, + [RD_KAFKAP_FetchSnapshot] = rd_true, + [RD_KAFKAP_BrokerHeartbeat] = rd_true, + [RD_KAFKAP_UnregisterBroker] = rd_true, + [RD_KAFKAP_AllocateProducerIds] = rd_true, }, [3 /*hide-unless-non-zero*/] = { /* Hide Admin requests unless they've been used */ - [RD_KAFKAP_CreateTopics] = rd_true, - [RD_KAFKAP_DeleteTopics] = rd_true, - [RD_KAFKAP_DeleteRecords] = rd_true, - [RD_KAFKAP_CreatePartitions] = rd_true, - [RD_KAFKAP_DescribeAcls] = rd_true, - [RD_KAFKAP_CreateAcls] = rd_true, - [RD_KAFKAP_DeleteAcls] = rd_true, - [RD_KAFKAP_DescribeConfigs] = rd_true, - [RD_KAFKAP_AlterConfigs] = rd_true, - [RD_KAFKAP_DeleteGroups] = rd_true, - [RD_KAFKAP_ListGroups] = rd_true, - [RD_KAFKAP_DescribeGroups] = rd_true}}; + [RD_KAFKAP_CreateTopics] = rd_true, + [RD_KAFKAP_DeleteTopics] = rd_true, + [RD_KAFKAP_DeleteRecords] = rd_true, + [RD_KAFKAP_CreatePartitions] = rd_true, + [RD_KAFKAP_DescribeAcls] = rd_true, + [RD_KAFKAP_CreateAcls] = rd_true, + [RD_KAFKAP_DeleteAcls] = rd_true, + [RD_KAFKAP_DescribeConfigs] = rd_true, + [RD_KAFKAP_AlterConfigs] = rd_true, + [RD_KAFKAP_DeleteGroups] = rd_true, + [RD_KAFKAP_ListGroups] = rd_true, + [RD_KAFKAP_DescribeGroups] = rd_true, + [RD_KAFKAP_DescribeLogDirs] = rd_true, + [RD_KAFKAP_IncrementalAlterConfigs] = rd_true, + [RD_KAFKAP_AlterPartitionReassignments] = rd_true, + [RD_KAFKAP_ListPartitionReassignments] = rd_true, + [RD_KAFKAP_OffsetDelete] = rd_true, + [RD_KAFKAP_DescribeClientQuotas] = rd_true, + [RD_KAFKAP_AlterClientQuotas] = rd_true, + [RD_KAFKAP_DescribeUserScramCredentials] = rd_true, + [RD_KAFKAP_AlterUserScramCredentials] = rd_true, + }}; int i; int cnt = 0; diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h index f5ae9ed753..396765857c 100644 --- a/src/rdkafka_proto.h +++ b/src/rdkafka_proto.h @@ -152,13 +152,21 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) { "DescribeUserScramCredentialsRequest", [RD_KAFKAP_AlterUserScramCredentials] = "AlterUserScramCredentialsRequest", - [RD_KAFKAP_Vote] = "VoteRequest", - [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", - [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", - [RD_KAFKAP_DescribeQuorum] = "DescribeQuorumRequest", - [RD_KAFKAP_AlterIsr] = "AlterIsrRequest", - [RD_KAFKAP_UpdateFeatures] = "UpdateFeaturesRequest", - [RD_KAFKAP_Envelope] = "EnvelopeRequest", + [RD_KAFKAP_Vote] = "VoteRequest", + [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", + [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", + [RD_KAFKAP_DescribeQuorum] = "DescribeQuorumRequest", + [RD_KAFKAP_AlterIsr] = "AlterIsrRequest", + [RD_KAFKAP_UpdateFeatures] = "UpdateFeaturesRequest", + [RD_KAFKAP_Envelope] = "EnvelopeRequest", + [RD_KAFKAP_FetchSnapshot] = "FetchSnapshot", + [RD_KAFKAP_DescribeCluster] = "DescribeCluster", + [RD_KAFKAP_DescribeProducers] = "DescribeProducers", + [RD_KAFKAP_BrokerHeartbeat] = "BrokerHeartbeat", + [RD_KAFKAP_UnregisterBroker] = "UnregisterBroker", + [RD_KAFKAP_DescribeTransactions] = "DescribeTransactions", + [RD_KAFKAP_ListTransactions] = "ListTransactions", + [RD_KAFKAP_AllocateProducerIds] = "AllocateProducerIds", }; static RD_TLS char ret[64]; diff --git a/src/rdkafka_protocol.h b/src/rdkafka_protocol.h index aa9db5392b..60c0999861 100644 --- a/src/rdkafka_protocol.h +++ b/src/rdkafka_protocol.h @@ -105,7 +105,16 @@ #define RD_KAFKAP_AlterIsr 56 #define RD_KAFKAP_UpdateFeatures 57 #define RD_KAFKAP_Envelope 58 -#define RD_KAFKAP__NUM 59 +#define RD_KAFKAP_FetchSnapshot 59 +#define RD_KAFKAP_DescribeCluster 60 +#define RD_KAFKAP_DescribeProducers 61 +#define RD_KAFKAP_BrokerHeartbeat 63 +#define RD_KAFKAP_UnregisterBroker 64 +#define RD_KAFKAP_DescribeTransactions 65 +#define RD_KAFKAP_ListTransactions 66 +#define RD_KAFKAP_AllocateProducerIds 67 + +#define RD_KAFKAP__NUM 68 #endif /* _RDKAFKA_PROTOCOL_H_ */ From 94292e81ea7962ea9b5671965a6c2119b33134c5 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Thu, 1 Dec 2022 20:46:33 +0100 Subject: [PATCH 03/40] Leader epoch support, WIP --- CHANGELOG.md | 10 ++++++++ STATISTICS.md | 3 +++ src-cpp/rdkafkacpp.h | 37 +++++++++++++++++++++++++++++- src-cpp/rdkafkacpp_int.h | 15 ++++++++++++ src/rdkafka.c | 17 ++++++++++---- src/rdkafka.h | 47 ++++++++++++++++++++++++++++++++++++++ src/rdkafka_partition.c | 9 ++++---- src/rdkafka_partition.h | 5 ++++ src/statistics_schema.json | 10 ++++++++ 9 files changed, 144 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c3e1889291..933bde5caa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -76,6 +76,16 @@ configuration property (refer [examples/rdkafka_complex_consumer_example.c] is now built-in. There are bundled builds with and without linking to cyrus-sasl for maximum compatibility. * Added `rd_kafka_topic_partition_get_leader_epoch()` (and `set..()`). + * Added partition leader epoch APIs: + - `rd_kafka_topic_partition_get_leader_epoch()` (and `set..()`) + - `rd_kafka_message_leader_epoch()` + - `rd_kafka_*assign()` and `rd_kafka_seek_partitions()` now supports + partitions with a leader epoch set. + - `rd_kafka_offsets_for_times()` will return per-partition leader-epochs. + - `leader_epoch`, `stored_leader_epoch`, and `committed_leader_epoch` + added to per-partition statistics. + + ## Fixes diff --git a/STATISTICS.md b/STATISTICS.md index 392e2cf05a..db2cb437b7 100644 --- a/STATISTICS.md +++ b/STATISTICS.md @@ -179,13 +179,16 @@ query_offset | int gauge | | Current/Last logical offset query next_offset | int gauge | | Next offset to fetch app_offset | int gauge | | Offset of last message passed to application + 1 stored_offset | int gauge | | Offset to be committed +stored_leader_epoch | int | | Partition leader epoch of stored offset committed_offset | int gauge | | Last committed offset +committed_leader_epoch | int | | Partition leader epoch of committed offset eof_offset | int gauge | | Last PARTITION_EOF signaled offset lo_offset | int gauge | | Partition's low watermark offset on broker hi_offset | int gauge | | Partition's high watermark offset on broker ls_offset | int gauge | | Partition's last stable offset on broker, or same as hi_offset is broker version is less than 0.11.0.0. consumer_lag | int gauge | | Difference between (hi_offset or ls_offset) and committed_offset). hi_offset is used when isolation.level=read_uncommitted, otherwise ls_offset. consumer_lag_stored | int gauge | | Difference between (hi_offset or ls_offset) and stored_offset. See consumer_lag and stored_offset. +leader_epoch | int | | Last known partition leader epoch, or -1 if unknown. txmsgs | int | | Total number of messages transmitted (produced) txbytes | int | | Total number of bytes transmitted for txmsgs rxmsgs | int | | Total number of messages consumed, not including ignored messages (due to offset, etc). diff --git a/src-cpp/rdkafkacpp.h b/src-cpp/rdkafkacpp.h index 7db203ee55..388b3c7d78 100644 --- a/src-cpp/rdkafkacpp.h +++ b/src-cpp/rdkafkacpp.h @@ -324,6 +324,8 @@ enum ErrorCode { ERR__NOOP = -141, /** No offset to automatically reset to */ ERR__AUTO_OFFSET_RESET = -140, + /** Partition log truncation detected */ + ERR__LOG_TRUNCATION = -139, /** End internal error codes */ ERR__END = -100, @@ -1982,7 +1984,7 @@ class RD_EXPORT TopicPartition { /** @brief Get partition leader epoch, or -1 if not known or relevant. */ virtual int32_t get_leader_epoch() = 0; - /* @brief Set partition leader epoch. */ + /** @brief Set partition leader epoch. */ virtual void set_leader_epoch(int32_t leader_epoch) = 0; }; @@ -2041,6 +2043,11 @@ class RD_EXPORT Topic { * The offset will be committed (written) to the broker (or file) according * to \p auto.commit.interval.ms or next manual offset-less commit call. * + * @deprecated This API lacks support for partition leader epochs, which makes + * it at risk for unclean leader election log truncation issues. + * Use KafkaConsumer::offsets_store() or + * Message::offset_store() instead. + * * @remark \c enable.auto.offset.store must be set to \c false when using * this API. * @@ -2471,6 +2478,31 @@ class RD_EXPORT Message { /** @returns the broker id of the broker the message was produced to or * fetched from, or -1 if not known/applicable. */ virtual int32_t broker_id() const = 0; + + /** @returns the message's partition leader epoch at the time the message was + * fetched and if known, else -1. */ + virtual int32_t leader_epoch() const = 0; + + /** + * @brief Store offset +1 for the consumed message. + * + * The message offset + 1 will be committed to broker according + * to \c `auto.commit.interval.ms` or manual offset-less commit() + * + * @warning This method may only be called for partitions that are currently + * assigned. + * Non-assigned partitions will fail with ERR__STATE. + * + * @warning Avoid storing offsets after calling seek() (et.al) as + * this may later interfere with resuming a paused partition, instead + * store offsets prior to calling seek. + * + * @remark \c `enable.auto.offset.store` must be set to "false" when using + * this API. + * + * @returns NULL on success or an error object on failure. + */ + virtual Error *offset_store() = 0; }; /**@}*/ @@ -2871,6 +2903,9 @@ class RD_EXPORT KafkaConsumer : public virtual Handle { * @remark \c enable.auto.offset.store must be set to \c false when using * this API. * + * @remark The leader epoch, if set, will be used to fence outdated partition + * leaders. See TopicPartition::set_leader_epoch(). + * * @returns RdKafka::ERR_NO_ERROR on success, or * RdKafka::ERR___UNKNOWN_PARTITION if none of the offsets could * be stored, or diff --git a/src-cpp/rdkafkacpp_int.h b/src-cpp/rdkafkacpp_int.h index 0cb23aa009..bc024ebe90 100644 --- a/src-cpp/rdkafkacpp_int.h +++ b/src-cpp/rdkafkacpp_int.h @@ -540,6 +540,21 @@ class MessageImpl : public Message { return rd_kafka_message_broker_id(rkmessage_); } + int32_t leader_epoch() const { + return rd_kafka_message_leader_epoch(rkmessage_); + } + + + Error *offset_store() { + rd_kafka_error_t *c_error; + + c_error = rd_kafka_offset_store_message(rkmessage_); + + if (c_error) + return new ErrorImpl(c_error); + else + return NULL; + } RdKafka::Topic *topic_; rd_kafka_message_t *rkmessage_; diff --git a/src/rdkafka.c b/src/rdkafka.c index 0d72a012b2..2c3a403682 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -489,6 +489,8 @@ static const struct rd_kafka_err_desc rd_kafka_err_descs[] = { _ERR_DESC(RD_KAFKA_RESP_ERR__NOOP, "Local: No operation performed"), _ERR_DESC(RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET, "Local: No offset to automatically reset to"), + _ERR_DESC(RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Local: Partition log truncation detected"), _ERR_DESC(RD_KAFKA_RESP_ERR_UNKNOWN, "Unknown broker error"), _ERR_DESC(RD_KAFKA_RESP_ERR_NO_ERROR, "Success"), @@ -1457,10 +1459,14 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st, ", " "\"stored_offset\":%" PRId64 ", " + "\"stored_leader_epoch\":%" PRId32 + ", " "\"commited_offset\":%" PRId64 ", " /*FIXME: issue #80 */ "\"committed_offset\":%" PRId64 ", " + "\"committed_leader_epoch\":%" PRId32 + ", " "\"eof_offset\":%" PRId64 ", " "\"lo_offset\":%" PRId64 @@ -1473,6 +1479,8 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st, ", " "\"consumer_lag_stored\":%" PRId64 ", " + "\"leader_epoch\":%" PRId32 + ", " "\"txmsgs\":%" PRIu64 ", " "\"txbytes\":%" PRIu64 @@ -1503,11 +1511,12 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st, rd_kafka_q_size(rktp->rktp_fetchq), rd_kafka_fetch_states[rktp->rktp_fetch_state], rktp->rktp_query_offset, offs.fetch_offset, rktp->rktp_app_offset, - rktp->rktp_stored_offset, + rktp->rktp_stored_offset, rktp->rktp_stored_leader_epoch, rktp->rktp_committed_offset, /* FIXME: issue #80 */ - rktp->rktp_committed_offset, offs.eof_offset, rktp->rktp_lo_offset, - rktp->rktp_hi_offset, rktp->rktp_ls_offset, consumer_lag, - consumer_lag_stored, rd_atomic64_get(&rktp->rktp_c.tx_msgs), + rktp->rktp_committed_offset, rktp->rktp_committed_leader_epoch, + offs.eof_offset, rktp->rktp_lo_offset, rktp->rktp_hi_offset, + rktp->rktp_ls_offset, consumer_lag, consumer_lag_stored, + rktp->rktp_leader_epoch, rd_atomic64_get(&rktp->rktp_c.tx_msgs), rd_atomic64_get(&rktp->rktp_c.tx_msg_bytes), rd_atomic64_get(&rktp->rktp_c.rx_msgs), rd_atomic64_get(&rktp->rktp_c.rx_msg_bytes), diff --git a/src/rdkafka.h b/src/rdkafka.h index b9c1decefe..ad8ef94305 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -402,6 +402,8 @@ typedef enum { RD_KAFKA_RESP_ERR__NOOP = -141, /** No offset to automatically reset to */ RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET = -140, + /** Partition log truncation detected */ + RD_KAFKA_RESP_ERR__LOG_TRUNCATION = -139, /** End internal error codes */ RD_KAFKA_RESP_ERR__END = -100, @@ -1614,6 +1616,18 @@ typedef enum { RD_EXPORT rd_kafka_msg_status_t rd_kafka_message_status(const rd_kafka_message_t *rkmessage); + +/** + * @returns the message's partition leader epoch at the time the message was + * fetched and if known, else -1. + * + * @remark This API must only be used on consumed messages without error. + * @remark Requires broker version >= 2.10 (KIP-320). + */ +RD_EXPORT int32_t +rd_kafka_message_leader_epoch(const rd_kafka_message_t *rkmessage); + + /**@}*/ @@ -3917,6 +3931,11 @@ int rd_kafka_consume_callback_queue( * The \c offset + 1 will be committed (written) to broker (or file) according * to \c `auto.commit.interval.ms` or manual offset-less commit() * + * @deprecated This API lacks support for partition leader epochs, which makes + * it at risk for unclean leader election log truncation issues. + * Use rd_kafka_offsets_store() and rd_kafka_offset_store_message() + * instead. + * * @warning This method may only be called for partitions that are currently * assigned. * Non-assigned partitions will fail with RD_KAFKA_RESP_ERR__STATE. @@ -3959,6 +3978,9 @@ rd_kafka_offset_store(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset); * @remark \c `enable.auto.offset.store` must be set to "false" when using * this API. * + * @remark The leader epoch, if set, will be used to fence outdated partition + * leaders. See rd_kafka_topic_partition_set_leader_epoch(). + * * @returns RD_KAFKA_RESP_ERR_NO_ERROR on (partial) success, or * RD_KAFKA_RESP_ERR__INVALID_ARG if \c enable.auto.offset.store * is true, or @@ -3968,6 +3990,31 @@ rd_kafka_offset_store(rd_kafka_topic_t *rkt, int32_t partition, int64_t offset); RD_EXPORT rd_kafka_resp_err_t rd_kafka_offsets_store(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *offsets); + + +/** + * @brief Store offset +1 for the consumed message. + * + * The message offset + 1 will be committed to broker according + * to \c `auto.commit.interval.ms` or manual offset-less commit() + * + * @warning This method may only be called for partitions that are currently + * assigned. + * Non-assigned partitions will fail with RD_KAFKA_RESP_ERR__STATE. + * Since v1.9.0. + * + * @warning Avoid storing offsets after calling rd_kafka_seek() (et.al) as + * this may later interfere with resuming a paused partition, instead + * store offsets prior to calling seek. + * + * @remark \c `enable.auto.offset.store` must be set to "false" when using + * this API. + * + * @returns NULL on success or an error object on failure. + */ +RD_EXPORT +rd_kafka_error_t *rd_kafka_offset_store_message(rd_kafka_message_t *rkmessage); + /**@}*/ diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index d61386412a..8208944edc 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -213,10 +213,11 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, rktp = rd_calloc(1, sizeof(*rktp)); - rktp->rktp_partition = partition; - rktp->rktp_rkt = rkt; - rktp->rktp_leader_id = -1; - rktp->rktp_broker_id = -1; + rktp->rktp_partition = partition; + rktp->rktp_rkt = rkt; + rktp->rktp_leader_id = -1; + rktp->rktp_broker_id = -1; + rktp->rktp_leader_epoch = -1; rd_interval_init(&rktp->rktp_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_log_intvl); diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 38c6e7f633..8f050a747a 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -242,6 +242,11 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ #define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \ ((fetch_state) >= RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY) + int32_t rktp_leader_epoch; /**< Last known partition leader epoch, + * or -1. */ + + + int32_t rktp_fetch_msg_max_bytes; /* Max number of bytes to * fetch. * Locality: broker thread diff --git a/src/statistics_schema.json b/src/statistics_schema.json index ab5c3d8016..d0dbedda7d 100644 --- a/src/statistics_schema.json +++ b/src/statistics_schema.json @@ -297,12 +297,19 @@ "stored_offset": { "type": "integer" }, + "stored_leader_epoch": { + "type": "integer" + }, "commited_offset": { "type": "integer" }, "committed_offset": { "type": "integer" }, + "committed_leader_epoch": { + "type": "integer" + }, + "eof_offset": { "type": "integer" }, @@ -318,6 +325,9 @@ "consumer_lag_stored": { "type": "integer" }, + "leader_epoch": { + "type": "integer" + }, "txmsgs": { "type": "integer" }, From cb7e6e2affd54e621e8b20d843c19abd42dbe6a0 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Tue, 6 Dec 2022 12:18:05 +0100 Subject: [PATCH 04/40] Test 0018: improve timeout robustness --- tests/0018-cgrp_term.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/0018-cgrp_term.c b/tests/0018-cgrp_term.c index baa72e2f2a..6b22339d7d 100644 --- a/tests/0018-cgrp_term.c +++ b/tests/0018-cgrp_term.c @@ -207,7 +207,7 @@ static void do_test(rd_bool_t with_queue) { test_conf_init(NULL, &default_topic_conf, - 5 + ((test_session_timeout_ms * 3) / 1000)); + 5 + ((test_session_timeout_ms * 3 * 2) / 1000)); if (rd_kafka_topic_conf_set(default_topic_conf, "auto.offset.reset", "smallest", errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) From b2e0652d3f605653760a92e88e7e8e20a18aadb2 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 21 Dec 2022 15:55:21 +0100 Subject: [PATCH 05/40] Replace consumer offsets with fetch_pos (offset, epoch) --- src/rdkafka.c | 46 ++-- src/rdkafka_assignment.c | 28 ++- src/rdkafka_cgrp.c | 8 +- src/rdkafka_fetcher.c | 114 +++++----- src/rdkafka_int.h | 11 + src/rdkafka_msgset_reader.c | 38 ++-- src/rdkafka_offset.c | 211 +++++++++++------- src/rdkafka_offset.h | 20 +- src/rdkafka_op.c | 10 +- src/rdkafka_op.h | 21 +- src/rdkafka_partition.c | 422 +++++++++++++++++++----------------- src/rdkafka_partition.h | 134 +++++++++--- src/rdkafka_request.c | 26 ++- 13 files changed, 666 insertions(+), 423 deletions(-) diff --git a/src/rdkafka.c b/src/rdkafka.c index 2c3a403682..2279887a62 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -1421,13 +1421,14 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st, * offsets are not (yet) committed. */ if (end_offset != RD_KAFKA_OFFSET_INVALID) { - if (rktp->rktp_stored_offset >= 0 && - rktp->rktp_stored_offset <= end_offset) + if (rktp->rktp_stored_pos.offset >= 0 && + rktp->rktp_stored_pos.offset <= end_offset) consumer_lag_stored = - end_offset - rktp->rktp_stored_offset; - if (rktp->rktp_committed_offset >= 0 && - rktp->rktp_committed_offset <= end_offset) - consumer_lag = end_offset - rktp->rktp_committed_offset; + end_offset - rktp->rktp_stored_pos.offset; + if (rktp->rktp_committed_pos.offset >= 0 && + rktp->rktp_committed_pos.offset <= end_offset) + consumer_lag = + end_offset - rktp->rktp_committed_pos.offset; } _st_printf( @@ -1510,13 +1511,15 @@ static RD_INLINE void rd_kafka_stats_emit_toppar(struct _stats_emit *st, 0, (size_t)0, rd_kafka_q_len(rktp->rktp_fetchq), rd_kafka_q_size(rktp->rktp_fetchq), rd_kafka_fetch_states[rktp->rktp_fetch_state], - rktp->rktp_query_offset, offs.fetch_offset, rktp->rktp_app_offset, - rktp->rktp_stored_offset, rktp->rktp_stored_leader_epoch, - rktp->rktp_committed_offset, /* FIXME: issue #80 */ - rktp->rktp_committed_offset, rktp->rktp_committed_leader_epoch, - offs.eof_offset, rktp->rktp_lo_offset, rktp->rktp_hi_offset, - rktp->rktp_ls_offset, consumer_lag, consumer_lag_stored, - rktp->rktp_leader_epoch, rd_atomic64_get(&rktp->rktp_c.tx_msgs), + rktp->rktp_query_pos.offset, offs.fetch_pos.offset, + rktp->rktp_app_pos.offset, rktp->rktp_stored_pos.offset, + rktp->rktp_stored_pos.leader_epoch, + rktp->rktp_committed_pos.offset, /* FIXME: issue #80 */ + rktp->rktp_committed_pos.offset, + rktp->rktp_committed_pos.leader_epoch, offs.eof_offset, + rktp->rktp_lo_offset, rktp->rktp_hi_offset, rktp->rktp_ls_offset, + consumer_lag, consumer_lag_stored, rktp->rktp_leader_epoch, + rd_atomic64_get(&rktp->rktp_c.tx_msgs), rd_atomic64_get(&rktp->rktp_c.tx_msg_bytes), rd_atomic64_get(&rktp->rktp_c.rx_msgs), rd_atomic64_get(&rktp->rktp_c.rx_msg_bytes), @@ -2722,7 +2725,8 @@ static RD_UNUSED int rd_kafka_consume_start0(rd_kafka_topic_t *rkt, return -1; } - rd_kafka_toppar_op_fetch_start(rktp, offset, rkq, RD_KAFKA_NO_REPLYQ); + rd_kafka_toppar_op_fetch_start(rktp, RD_KAFKA_FETCH_POS(offset, -1), + rkq, RD_KAFKA_NO_REPLYQ); rd_kafka_toppar_destroy(rktp); @@ -2834,7 +2838,8 @@ rd_kafka_resp_err_t rd_kafka_seek(rd_kafka_topic_t *app_rkt, replyq = RD_KAFKA_REPLYQ(tmpq, 0); } - if ((err = rd_kafka_toppar_op_seek(rktp, offset, replyq))) { + if ((err = rd_kafka_toppar_op_seek(rktp, RD_KAFKA_FETCH_POS(offset, -1), + replyq))) { if (tmpq) rd_kafka_q_destroy_owner(tmpq); rd_kafka_toppar_destroy(rktp); @@ -2886,8 +2891,9 @@ rd_kafka_seek_partitions(rd_kafka_t *rk, continue; } - err = rd_kafka_toppar_op_seek(rktp, rktpar->offset, - RD_KAFKA_REPLYQ(tmpq, 0)); + err = rd_kafka_toppar_op_seek( + rktp, rd_kafka_topic_partition_get_fetch_pos(rktpar), + RD_KAFKA_REPLYQ(tmpq, 0)); if (err) { rktpar->err = err; } else { @@ -3440,10 +3446,12 @@ rd_kafka_position(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *partitions) { } rd_kafka_toppar_lock(rktp); - rktpar->offset = rktp->rktp_app_offset; - rktpar->err = RD_KAFKA_RESP_ERR_NO_ERROR; + rd_kafka_topic_partition_set_from_fetch_pos(rktpar, + rktp->rktp_app_pos); rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); + + rktpar->err = RD_KAFKA_RESP_ERR_NO_ERROR; } return RD_KAFKA_RESP_ERR_NO_ERROR; diff --git a/src/rdkafka_assignment.c b/src/rdkafka_assignment.c index ea0d533af9..0ddcecb3ce 100644 --- a/src/rdkafka_assignment.c +++ b/src/rdkafka_assignment.c @@ -337,17 +337,19 @@ static int rd_kafka_assignment_serve_removals(rd_kafka_t *rk) { rd_kafka_toppar_lock(rktp); - /* Save the currently stored offset on .removed + /* Save the currently stored offset and epoch on .removed * so it will be committed below. */ - rktpar->offset = rktp->rktp_stored_offset; + rd_kafka_topic_partition_set_from_fetch_pos( + rktpar, rktp->rktp_stored_pos); valid_offsets += !RD_KAFKA_OFFSET_IS_LOGICAL(rktpar->offset); /* Reset the stored offset to invalid so that * a manual offset-less commit() or the auto-committer * will not commit a stored offset from a previous * assignment (issue #2782). */ - rd_kafka_offset_store0(rktp, RD_KAFKA_OFFSET_INVALID, rd_true, - RD_DONT_LOCK); + rd_kafka_offset_store0( + rktp, RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), + rd_true, RD_DONT_LOCK); /* Partition is no longer desired */ rd_kafka_toppar_desired_del(rktp); @@ -449,9 +451,11 @@ static int rd_kafka_assignment_serve_pending(rd_kafka_t *rk) { rd_kafka_dbg(rk, CGRP, "SRVPEND", "Starting pending assigned partition " - "%s [%" PRId32 "] at offset %s", + "%s [%" PRId32 "] at %s", rktpar->topic, rktpar->partition, - rd_kafka_offset2str(rktpar->offset)); + rd_kafka_fetch_pos2str( + rd_kafka_topic_partition_get_fetch_pos( + rktpar))); /* Reset the (lib) pause flag which may have been set by * the cgrp when scheduling the rebalance callback. */ @@ -463,9 +467,10 @@ static int rd_kafka_assignment_serve_pending(rd_kafka_t *rk) { rktp->rktp_started = rd_true; rk->rk_consumer.assignment.started_cnt++; - rd_kafka_toppar_op_fetch_start(rktp, rktpar->offset, - rk->rk_consumer.q, - RD_KAFKA_NO_REPLYQ); + rd_kafka_toppar_op_fetch_start( + rktp, + rd_kafka_topic_partition_get_fetch_pos(rktpar), + rk->rk_consumer.q, RD_KAFKA_NO_REPLYQ); } else if (can_query_offsets) { @@ -738,8 +743,9 @@ rd_kafka_assignment_add(rd_kafka_t *rk, /* Reset the stored offset to INVALID to avoid the race * condition described in rdkafka_offset.h */ - rd_kafka_offset_store0(rktp, RD_KAFKA_OFFSET_INVALID, - rd_true /* force */, RD_DONT_LOCK); + rd_kafka_offset_store0( + rktp, RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), + rd_true /* force */, RD_DONT_LOCK); rd_kafka_toppar_unlock(rktp); } diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 1e5e10c610..b3f70444ff 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -2832,7 +2832,8 @@ static int rd_kafka_cgrp_update_committed_offsets( continue; rd_kafka_toppar_lock(rktp); - rktp->rktp_committed_offset = rktpar->offset; + rktp->rktp_committed_pos = + rd_kafka_topic_partition_get_fetch_pos(rktpar); rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* from get_toppar() */ @@ -3076,8 +3077,9 @@ static size_t rd_kafka_topic_partition_has_absolute_offset( * * \p rko...silent_empty: if there are no offsets to commit bail out * silently without posting an op on the reply queue. - * \p set_offsets: set offsets in rko->rko_u.offset_commit.partitions from - * the rktp's stored offset. + * \p set_offsets: set offsets and epochs in + * rko->rko_u.offset_commit.partitions from the rktp's + * stored offset. * * Locality: cgrp thread */ diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 5003e3d8df..5cc40c6c64 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -193,33 +193,38 @@ static void rd_kafka_fetch_reply_handle_partition_error( * - HWM is >= offset, but msg not * yet available at that offset * (replica is out of sync). + * - partition leader is out of sync. * - * Handle by retrying FETCH (with backoff). + * Handle by requesting metadata update and + * retrying FETCH (with backoff). */ rd_rkb_dbg(rkb, MSG, "FETCH", - "Topic %s [%" PRId32 "]: Offset %" PRId64 - " not " + "Topic %s [%" PRId32 + "]: %s not " "available on broker %" PRId32 " (leader %" PRId32 - "): retrying", + "): updating metadata and retrying", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rktp->rktp_offsets.fetch_offset, + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), rktp->rktp_broker_id, rktp->rktp_leader_id); + rd_kafka_topic_fast_leader_query(rkb->rkb_rk); break; case RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE: { - int64_t err_offset; + rd_kafka_fetch_pos_t err_pos; if (rktp->rktp_broker_id != rktp->rktp_leader_id && - rktp->rktp_offsets.fetch_offset > HighwaterMarkOffset) { + rktp->rktp_offsets.fetch_pos.offset > HighwaterMarkOffset) { rd_kafka_log(rkb->rkb_rk, LOG_WARNING, "FETCH", - "Topic %s [%" PRId32 "]: Offset %" PRId64 + "Topic %s [%" PRId32 + "]: %s " " out of range (HighwaterMark %" PRId64 " fetching from " "broker %" PRId32 " (leader %" PRId32 "): reverting to leader", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rktp->rktp_offsets.fetch_offset, + rd_kafka_fetch_pos2str( + rktp->rktp_offsets.fetch_pos), HighwaterMarkOffset, rktp->rktp_broker_id, rktp->rktp_leader_id); @@ -232,9 +237,10 @@ static void rd_kafka_fetch_reply_handle_partition_error( } /* Application error */ - err_offset = rktp->rktp_offsets.fetch_offset; - rktp->rktp_offsets.fetch_offset = RD_KAFKA_OFFSET_INVALID; - rd_kafka_offset_reset(rktp, rd_kafka_broker_id(rkb), err_offset, + err_pos = rktp->rktp_offsets.fetch_pos; + rktp->rktp_offsets.fetch_pos.offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_offsets.fetch_pos.leader_epoch = -1; + rd_kafka_offset_reset(rktp, rd_kafka_broker_id(rkb), err_pos, err, "fetch failed due to requested offset " "not available on the broker"); @@ -248,7 +254,7 @@ static void rd_kafka_fetch_reply_handle_partition_error( rd_kafka_consumer_err( rktp->rktp_fetchq, rd_kafka_broker_id(rkb), err, tver->version, NULL, rktp, - rktp->rktp_offsets.fetch_offset, + rktp->rktp_offsets.fetch_pos.offset, "Fetch from broker %" PRId32 " failed: %s", rd_kafka_broker_id(rkb), rd_kafka_err2str(err)); rktp->rktp_last_error = err; @@ -259,17 +265,17 @@ static void rd_kafka_fetch_reply_handle_partition_error( /* Application errors */ case RD_KAFKA_RESP_ERR__PARTITION_EOF: if (rkb->rkb_rk->rk_conf.enable_partition_eof) - rd_kafka_consumer_err(rktp->rktp_fetchq, - rd_kafka_broker_id(rkb), err, - tver->version, NULL, rktp, - rktp->rktp_offsets.fetch_offset, - "Fetch from broker %" PRId32 - " reached end of " - "partition at offset %" PRId64 - " (HighwaterMark %" PRId64 ")", - rd_kafka_broker_id(rkb), - rktp->rktp_offsets.fetch_offset, - HighwaterMarkOffset); + rd_kafka_consumer_err( + rktp->rktp_fetchq, rd_kafka_broker_id(rkb), err, + tver->version, NULL, rktp, + rktp->rktp_offsets.fetch_pos.offset, + "Fetch from broker %" PRId32 + " reached end of " + "partition at offset %" PRId64 + " (HighwaterMark %" PRId64 ")", + rd_kafka_broker_id(rkb), + rktp->rktp_offsets.fetch_pos.offset, + HighwaterMarkOffset); break; case RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE: @@ -277,9 +283,12 @@ static void rd_kafka_fetch_reply_handle_partition_error( rd_dassert(tver->version > 0); rd_kafka_consumer_err( rktp->rktp_fetchq, rd_kafka_broker_id(rkb), err, - tver->version, NULL, rktp, rktp->rktp_offsets.fetch_offset, - "Fetch from broker %" PRId32 " failed: %s", - rd_kafka_broker_id(rkb), rd_kafka_err2str(err)); + tver->version, NULL, rktp, + rktp->rktp_offsets.fetch_pos.offset, + "Fetch from broker %" PRId32 " failed at %s: %s", + rd_kafka_broker_id(rkb), + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), + rd_kafka_err2str(err)); break; } @@ -508,10 +517,10 @@ rd_kafka_fetch_reply_handle_partition(rd_kafka_broker_t *rkb, /* If this is the last message of the queue, * signal EOF back to the application. */ - if (end_offset == rktp->rktp_offsets.fetch_offset && - rktp->rktp_offsets.eof_offset != rktp->rktp_offsets.fetch_offset) { + if (end_offset == rktp->rktp_offsets.fetch_pos.offset && + rktp->rktp_offsets.eof_offset != end_offset) { hdr.ErrorCode = RD_KAFKA_RESP_ERR__PARTITION_EOF; - rktp->rktp_offsets.eof_offset = rktp->rktp_offsets.fetch_offset; + rktp->rktp_offsets.eof_offset = end_offset; } if (unlikely(hdr.ErrorCode != RD_KAFKA_RESP_ERR_NO_ERROR)) { @@ -831,7 +840,8 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { rd_kafka_buf_write_i32(rkbuf, -1); /* FetchOffset */ - rd_kafka_buf_write_i64(rkbuf, rktp->rktp_offsets.fetch_offset); + rd_kafka_buf_write_i64(rkbuf, + rktp->rktp_offsets.fetch_pos.offset); if (rd_kafka_buf_ApiVersion(rkbuf) >= 5) /* LogStartOffset - only used by follower replica */ @@ -842,14 +852,15 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { rd_rkb_dbg(rkb, FETCH, "FETCH", "Fetch topic %.*s [%" PRId32 "] at offset %" PRId64 - " (v%d)", + " (leader epoch %" PRId32 ", v%d)", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, - rktp->rktp_offsets.fetch_offset, + rktp->rktp_offsets.fetch_pos.offset, + rktp->rktp_offsets.fetch_pos.leader_epoch, rktp->rktp_fetch_version); /* We must have a valid fetch offset when we get here */ - rd_dassert(rktp->rktp_offsets.fetch_offset >= 0); + rd_dassert(rktp->rktp_offsets.fetch_pos.offset >= 0); /* Add toppar + op version mapping. */ tver = rd_list_add(rkbuf->rkbuf_rktp_vers, NULL); @@ -976,8 +987,9 @@ rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, /* Update broker thread's fetch op version */ version = rktp->rktp_op_version; if (version > rktp->rktp_fetch_version || - rktp->rktp_next_offset != rktp->rktp_last_next_offset || - rktp->rktp_offsets.fetch_offset == RD_KAFKA_OFFSET_INVALID) { + rd_kafka_fetch_pos_cmp(&rktp->rktp_next_fetch_start, + &rktp->rktp_last_next_fetch_start) || + rktp->rktp_offsets.fetch_pos.offset == RD_KAFKA_OFFSET_INVALID) { /* New version barrier, something was modified from the * control plane. Reset and start over. * Alternatively only the next_offset changed but not the @@ -985,21 +997,22 @@ rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, * offset.reset (such as on PARTITION_EOF or * OFFSET_OUT_OF_RANGE). */ - rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "FETCHDEC", - "Topic %s [%" PRId32 - "]: fetch decide: " - "updating to version %d (was %d) at " - "offset %" PRId64 " (was %" PRId64 ")", - rktp->rktp_rkt->rkt_topic->str, - rktp->rktp_partition, version, - rktp->rktp_fetch_version, rktp->rktp_next_offset, - rktp->rktp_offsets.fetch_offset); + rd_kafka_dbg( + rktp->rktp_rkt->rkt_rk, TOPIC, "FETCHDEC", + "Topic %s [%" PRId32 + "]: fetch decide: " + "updating to version %d (was %d) at %s " + "(was %s)", + rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, + version, rktp->rktp_fetch_version, + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos)); rd_kafka_offset_stats_reset(&rktp->rktp_offsets); /* New start offset */ - rktp->rktp_offsets.fetch_offset = rktp->rktp_next_offset; - rktp->rktp_last_next_offset = rktp->rktp_next_offset; + rktp->rktp_offsets.fetch_pos = rktp->rktp_next_fetch_start; + rktp->rktp_last_next_fetch_start = rktp->rktp_next_fetch_start; rktp->rktp_fetch_version = version; @@ -1016,7 +1029,8 @@ rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, should_fetch = 0; reason = "paused"; - } else if (RD_KAFKA_OFFSET_IS_LOGICAL(rktp->rktp_next_offset)) { + } else if (RD_KAFKA_OFFSET_IS_LOGICAL( + rktp->rktp_next_fetch_start.offset)) { should_fetch = 0; reason = "no concrete offset"; @@ -1046,13 +1060,13 @@ rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, rd_rkb_dbg( rkb, FETCH, "FETCH", "Topic %s [%" PRId32 - "] in state %s at offset %s " + "] in state %s at %s " "(%d/%d msgs, %" PRId64 "/%d kb queued, " "opv %" PRId32 ") is %s%s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rd_kafka_fetch_states[rktp->rktp_fetch_state], - rd_kafka_offset2str(rktp->rktp_next_offset), + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), rd_kafka_q_len(rktp->rktp_fetchq), rkb->rkb_rk->rk_conf.queued_min_msgs, rd_kafka_q_size(rktp->rktp_fetchq) / 1024, diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h index e663b34856..d65978fdb2 100644 --- a/src/rdkafka_int.h +++ b/src/rdkafka_int.h @@ -82,6 +82,17 @@ struct rd_kafka_toppar_s; typedef struct rd_kafka_lwtopic_s rd_kafka_lwtopic_t; +/** + * @struct Represents a fetch position: + * an offset and an partition leader epoch (if known, else -1). + */ +typedef struct rd_kafka_fetch_pos_s { + int64_t offset; + int32_t leader_epoch; +} rd_kafka_fetch_pos_t; + + + #include "rdkafka_op.h" #include "rdkafka_queue.h" #include "rdkafka_msg.h" diff --git a/src/rdkafka_msgset_reader.c b/src/rdkafka_msgset_reader.c index 02a4c02f85..bcad9b0578 100644 --- a/src/rdkafka_msgset_reader.c +++ b/src/rdkafka_msgset_reader.c @@ -168,6 +168,9 @@ typedef struct rd_kafka_msgset_reader_s { const struct rd_kafka_toppar_ver *msetr_tver; /**< Toppar op version of * request. */ + int32_t msetr_leader_epoch; /**< Current MessageSet's partition + * leader epoch (or -1). */ + int32_t msetr_broker_id; /**< Broker id (of msetr_rkb) */ rd_kafka_broker_t *msetr_rkb; /* @warning Not a refcounted * reference! */ @@ -230,6 +233,7 @@ static void rd_kafka_msgset_reader_init(rd_kafka_msgset_reader_t *msetr, memset(msetr, 0, sizeof(*msetr)); msetr->msetr_rkb = rkbuf->rkbuf_rkb; + msetr->msetr_leader_epoch = -1; msetr->msetr_broker_id = rd_kafka_broker_id(msetr->msetr_rkb); msetr->msetr_rktp = rktp; msetr->msetr_aborted_txns = aborted_txns; @@ -647,7 +651,8 @@ rd_kafka_msgset_reader_msg_v0_1(rd_kafka_msgset_reader_t *msetr) { * the messageset, and it also means * we cant perform this offset check here * in that case. */ - if (!relative_offsets && hdr.Offset < rktp->rktp_offsets.fetch_offset) + if (!relative_offsets && + hdr.Offset < rktp->rktp_offsets.fetch_pos.offset) return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue with next msg */ /* Handle compressed MessageSet */ @@ -669,7 +674,8 @@ rd_kafka_msgset_reader_msg_v0_1(rd_kafka_msgset_reader_t *msetr) { (size_t)RD_KAFKAP_BYTES_LEN(&Value), RD_KAFKAP_BYTES_IS_NULL(&Value) ? NULL : Value.data); - rkm->rkm_broker_id = msetr->msetr_broker_id; + rkm->rkm_u.consumer.leader_epoch = msetr->msetr_leader_epoch; + rkm->rkm_broker_id = msetr->msetr_broker_id; /* Assign message timestamp. * If message was in a compressed MessageSet and the outer/wrapper @@ -738,13 +744,13 @@ rd_kafka_msgset_reader_msg_v2(rd_kafka_msgset_reader_t *msetr) { hdr.Offset = msetr->msetr_v2_hdr->BaseOffset + hdr.OffsetDelta; /* Skip message if outdated */ - if (hdr.Offset < rktp->rktp_offsets.fetch_offset) { + if (hdr.Offset < rktp->rktp_offsets.fetch_pos.offset) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "MSG", "%s [%" PRId32 "]: " "Skip offset %" PRId64 " < fetch_offset %" PRId64, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - hdr.Offset, rktp->rktp_offsets.fetch_offset); + hdr.Offset, rktp->rktp_offsets.fetch_pos.offset); rd_kafka_buf_skip_to(rkbuf, message_end); return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue with next msg */ } @@ -905,7 +911,8 @@ rd_kafka_msgset_reader_msg_v2(rd_kafka_msgset_reader_t *msetr) { (size_t)RD_KAFKAP_BYTES_LEN(&hdr.Value), RD_KAFKAP_BYTES_IS_NULL(&hdr.Value) ? NULL : hdr.Value.data); - rkm->rkm_broker_id = msetr->msetr_broker_id; + rkm->rkm_u.consumer.leader_epoch = msetr->msetr_leader_epoch; + rkm->rkm_broker_id = msetr->msetr_broker_id; /* Store pointer to unparsed message headers, they will * be parsed on the first access. @@ -1045,6 +1052,8 @@ rd_kafka_msgset_reader_v2(rd_kafka_msgset_reader_t *msetr) { RD_KAFKAP_MSGSET_V2_SIZE - 8 - 4); rd_kafka_buf_read_i32(rkbuf, &hdr.PartitionLeaderEpoch); + msetr->msetr_leader_epoch = hdr.PartitionLeaderEpoch; + rd_kafka_buf_read_i8(rkbuf, &hdr.MagicByte); rd_kafka_buf_read_i32(rkbuf, &hdr.Crc); @@ -1105,7 +1114,7 @@ rd_kafka_msgset_reader_v2(rd_kafka_msgset_reader_t *msetr) { hdr.BaseOffset, payload_size); /* If entire MessageSet contains old outdated offsets, skip it. */ - if (LastOffset < rktp->rktp_offsets.fetch_offset) { + if (LastOffset < rktp->rktp_offsets.fetch_pos.offset) { rd_kafka_buf_skip(rkbuf, payload_size); goto done; } @@ -1215,7 +1224,8 @@ rd_kafka_msgset_reader_peek_msg_version(rd_kafka_msgset_reader_t *msetr, (int)*MagicBytep, Offset, read_offset, rd_slice_size(&rkbuf->rkbuf_reader)); - if (Offset >= msetr->msetr_rktp->rktp_offsets.fetch_offset) { + if (Offset >= + msetr->msetr_rktp->rktp_offsets.fetch_pos.offset) { rd_kafka_consumer_err( &msetr->msetr_rkq, msetr->msetr_broker_id, RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED, @@ -1224,7 +1234,7 @@ rd_kafka_msgset_reader_peek_msg_version(rd_kafka_msgset_reader_t *msetr, "at offset %" PRId64, (int)*MagicBytep, Offset); /* Skip message(set) */ - msetr->msetr_rktp->rktp_offsets.fetch_offset = + msetr->msetr_rktp->rktp_offsets.fetch_pos.offset = Offset + 1; } @@ -1311,7 +1321,7 @@ static void rd_kafka_msgset_reader_postproc(rd_kafka_msgset_reader_t *msetr, * fetch offset. */ rd_kafka_q_fix_offsets( &msetr->msetr_rkq, - msetr->msetr_rktp->rktp_offsets.fetch_offset, + msetr->msetr_rktp->rktp_offsets.fetch_pos.offset, msetr->msetr_outer.offset - *last_offsetp); } } @@ -1376,11 +1386,11 @@ rd_kafka_msgset_reader_run(rd_kafka_msgset_reader_t *msetr) { &msetr->msetr_rkq, msetr->msetr_broker_id, RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE, msetr->msetr_tver->version, NULL, rktp, - rktp->rktp_offsets.fetch_offset, + rktp->rktp_offsets.fetch_pos.offset, "Message at offset %" PRId64 " might be too large to fetch, try increasing " "receive.message.max.bytes", - rktp->rktp_offsets.fetch_offset); + rktp->rktp_offsets.fetch_pos.offset); } else if (msetr->msetr_aborted_cnt > 0) { /* Noop */ @@ -1421,13 +1431,13 @@ rd_kafka_msgset_reader_run(rd_kafka_msgset_reader_t *msetr) { /* Update partition's fetch offset based on * last message's offest. */ if (likely(last_offset != -1)) - rktp->rktp_offsets.fetch_offset = last_offset + 1; + rktp->rktp_offsets.fetch_pos.offset = last_offset + 1; } /* Adjust next fetch offset if outlier code has indicated * an even later next offset. */ - if (msetr->msetr_next_offset > rktp->rktp_offsets.fetch_offset) - rktp->rktp_offsets.fetch_offset = msetr->msetr_next_offset; + if (msetr->msetr_next_offset > rktp->rktp_offsets.fetch_pos.offset) + rktp->rktp_offsets.fetch_pos.offset = msetr->msetr_next_offset; rd_kafka_q_destroy_owner(&msetr->msetr_rkq); diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 805da2d18b..615905faf2 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -261,7 +261,7 @@ rd_kafka_offset_file_commit(rd_kafka_toppar_t *rktp) { rd_kafka_topic_t *rkt = rktp->rktp_rkt; int attempt; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; - int64_t offset = rktp->rktp_stored_offset; + int64_t offset = rktp->rktp_stored_pos.offset; for (attempt = 0; attempt < 2; attempt++) { char buf[22]; @@ -322,7 +322,7 @@ rd_kafka_offset_file_commit(rd_kafka_toppar_t *rktp) { rktp->rktp_partition, offset, rktp->rktp_offset_path); - rktp->rktp_committed_offset = offset; + rktp->rktp_committed_pos.offset = offset; /* If sync interval is set to immediate we sync right away. */ if (rkt->rkt_conf.offset_store_sync_interval_ms == 0) @@ -528,7 +528,7 @@ rd_kafka_offset_broker_commit_cb(rd_kafka_t *rk, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktpar->offset, err ? "not " : "", rd_kafka_err2str(err)); - rktp->rktp_committing_offset = 0; + rktp->rktp_committing_pos.offset = 0; rd_kafka_toppar_lock(rktp); if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE_STOPPING) @@ -539,6 +539,9 @@ rd_kafka_offset_broker_commit_cb(rd_kafka_t *rk, } +/** + * @locks_required rd_kafka_toppar_lock(rktp) MUST be held. + */ static rd_kafka_resp_err_t rd_kafka_offset_broker_commit(rd_kafka_toppar_t *rktp, const char *reason) { rd_kafka_topic_partition_list_t *offsets; @@ -548,18 +551,19 @@ rd_kafka_offset_broker_commit(rd_kafka_toppar_t *rktp, const char *reason) { rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE); - rktp->rktp_committing_offset = rktp->rktp_stored_offset; + rktp->rktp_committing_pos = rktp->rktp_stored_pos; offsets = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); - rktpar->offset = rktp->rktp_committing_offset; + rd_kafka_topic_partition_set_from_fetch_pos(rktpar, + rktp->rktp_committing_pos); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSETCMT", - "%.*s [%" PRId32 "]: committing offset %" PRId64 ": %s", + "%.*s [%" PRId32 "]: committing %s: %s", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rktp->rktp_committing_offset, - reason); + rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_committing_pos), reason); rd_kafka_commit0(rktp->rktp_rkt->rkt_rk, offsets, rktp, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0), @@ -580,21 +584,20 @@ rd_kafka_offset_broker_commit(rd_kafka_toppar_t *rktp, const char *reason) { */ static rd_kafka_resp_err_t rd_kafka_offset_commit(rd_kafka_toppar_t *rktp, const char *reason) { - if (1) // FIXME - rd_kafka_dbg( - rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", - "%s [%" PRId32 - "]: commit: " - "stored offset %" PRId64 " > committed offset %" PRId64 "?", - rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rktp->rktp_stored_offset, rktp->rktp_committed_offset); + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", + "%s [%" PRId32 "]: commit: stored %s > committed %s?", + rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_stored_pos), + rd_kafka_fetch_pos2str(rktp->rktp_committed_pos)); /* Already committed */ - if (rktp->rktp_stored_offset <= rktp->rktp_committed_offset) + if (rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos, + &rktp->rktp_committed_pos) <= 0) return RD_KAFKA_RESP_ERR_NO_ERROR; /* Already committing (for async ops) */ - if (rktp->rktp_stored_offset <= rktp->rktp_committing_offset) + if (rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos, + &rktp->rktp_committing_pos) <= 0) return RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS; switch (rktp->rktp_rkt->rkt_conf.offset_store_method) { @@ -630,6 +633,8 @@ rd_kafka_resp_err_t rd_kafka_offset_sync(rd_kafka_toppar_t *rktp) { * Typically called from application code. * * NOTE: No locks must be held. + * + * @deprecated Use rd_kafka_offsets_store(). */ rd_kafka_resp_err_t rd_kafka_offset_store(rd_kafka_topic_t *app_rkt, int32_t partition, @@ -637,6 +642,7 @@ rd_kafka_resp_err_t rd_kafka_offset_store(rd_kafka_topic_t *app_rkt, rd_kafka_topic_t *rkt = rd_kafka_topic_proper(app_rkt); rd_kafka_toppar_t *rktp; rd_kafka_resp_err_t err; + rd_kafka_fetch_pos_t pos = {offset + 1, -1 /*no leader epoch known*/}; /* Find toppar */ rd_kafka_topic_rdlock(rkt); @@ -646,8 +652,8 @@ rd_kafka_resp_err_t rd_kafka_offset_store(rd_kafka_topic_t *app_rkt, } rd_kafka_topic_rdunlock(rkt); - err = rd_kafka_offset_store0(rktp, offset + 1, - rd_false /* Don't force */, RD_DO_LOCK); + err = rd_kafka_offset_store0(rktp, pos, rd_false /* Don't force */, + RD_DO_LOCK); rd_kafka_toppar_destroy(rktp); @@ -668,6 +674,7 @@ rd_kafka_offsets_store(rd_kafka_t *rk, for (i = 0; i < offsets->cnt; i++) { rd_kafka_topic_partition_t *rktpar = &offsets->elems[i]; rd_kafka_toppar_t *rktp; + rd_kafka_fetch_pos_t pos = {rktpar->offset, -1}; rktp = rd_kafka_topic_partition_get_toppar(rk, rktpar, rd_false); @@ -677,9 +684,11 @@ rd_kafka_offsets_store(rd_kafka_t *rk, continue; } - rktpar->err = rd_kafka_offset_store0(rktp, rktpar->offset, - rd_false /* don't force */, - RD_DO_LOCK); + pos.leader_epoch = + rd_kafka_topic_partition_get_leader_epoch(rktpar); + + rktpar->err = rd_kafka_offset_store0( + rktp, pos, rd_false /* don't force */, RD_DO_LOCK); rd_kafka_toppar_destroy(rktp); if (rktpar->err) @@ -693,6 +702,39 @@ rd_kafka_offsets_store(rd_kafka_t *rk, } +rd_kafka_error_t *rd_kafka_offset_store_message(rd_kafka_message_t *rkmessage) { + rd_kafka_toppar_t *rktp; + rd_kafka_op_t *rko; + rd_kafka_resp_err_t err; + rd_kafka_msg_t *rkm = (rd_kafka_msg_t *)rkmessage; + rd_kafka_fetch_pos_t pos; + + if (rkmessage->err) + return rd_kafka_error_new(RD_KAFKA_RESP_ERR__INVALID_ARG, + "Message object must not have an " + "error set"); + + if (unlikely(!(rko = rd_kafka_message2rko(rkmessage)) || + !(rktp = rko->rko_rktp))) + return rd_kafka_error_new(RD_KAFKA_RESP_ERR__INVALID_ARG, + "Invalid message object, " + "not a consumed message"); + + pos.offset = rkmessage->offset + 1; + pos.leader_epoch = rkm->rkm_u.consumer.leader_epoch; + err = rd_kafka_offset_store0(rktp, pos, rd_false /* Don't force */, + RD_DO_LOCK); + + if (err == RD_KAFKA_RESP_ERR__STATE) + return rd_kafka_error_new(err, "Partition is not assigned"); + else if (err) + return rd_kafka_error_new(err, "Failed to store offset: %s", + rd_kafka_err2str(err)); + + return NULL; +} + + /** * Decommissions the use of an offset file for a toppar. @@ -723,7 +765,7 @@ static rd_kafka_op_res_t rd_kafka_offset_reset_op_cb(rd_kafka_t *rk, rd_kafka_toppar_t *rktp = rko->rko_rktp; rd_kafka_toppar_lock(rktp); rd_kafka_offset_reset(rktp, rko->rko_u.offset_reset.broker_id, - rko->rko_u.offset_reset.offset, rko->rko_err, + rko->rko_u.offset_reset.pos, rko->rko_err, "%s", rko->rko_u.offset_reset.reason); rd_kafka_toppar_unlock(rktp); return RD_KAFKA_OP_RES_HANDLED; @@ -735,20 +777,29 @@ static rd_kafka_op_res_t rd_kafka_offset_reset_op_cb(rd_kafka_t *rk, * * @param rktp the toppar * @param broker_id Originating broker, if any, else RD_KAFKA_NODEID_UA. - * @param err_offset a logical offset, or offset corresponding to the error. + * @param err_pos a logical offset, or offset corresponding to the error. * @param err the error, or RD_KAFKA_RESP_ERR_NO_ERROR if offset is logical. - * @param reason a reason string for logging. + * @param fmt a reason string for logging. * - * @locality: any. if not main thread, work will be enqued on main thread. - * @ocks: toppar_lock() MUST be held + * @locality any. if not main thread, work will be enqued on main thread. + * @locks_required toppar_lock() MUST be held */ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, int32_t broker_id, - int64_t err_offset, + rd_kafka_fetch_pos_t err_pos, rd_kafka_resp_err_t err, - const char *reason) { - int64_t offset = RD_KAFKA_OFFSET_INVALID; - const char *extra = ""; + const char *fmt, + ...) { + rd_kafka_fetch_pos_t pos = {RD_KAFKA_OFFSET_INVALID, -1}; + const char *extra = ""; + char reason[512]; + va_list ap; + + va_start(ap, fmt); + rd_vsnprintf(reason, sizeof(reason), fmt, ap); + va_end(ap); + + rd_assert(err != RD_KAFKA_RESP_ERR__LOG_TRUNCATION); /* Enqueue op for toppar handler thread if we're on the wrong thread. */ if (!thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)) { @@ -758,48 +809,49 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, rko->rko_err = err; rko->rko_rktp = rd_kafka_toppar_keep(rktp); rko->rko_u.offset_reset.broker_id = broker_id; - rko->rko_u.offset_reset.offset = err_offset; + rko->rko_u.offset_reset.pos = err_pos; rko->rko_u.offset_reset.reason = rd_strdup(reason); rd_kafka_q_enq(rktp->rktp_ops, rko); return; } - if (err_offset == RD_KAFKA_OFFSET_INVALID || err) - offset = rktp->rktp_rkt->rkt_conf.auto_offset_reset; + if (err_pos.offset == RD_KAFKA_OFFSET_INVALID || err) + pos.offset = rktp->rktp_rkt->rkt_conf.auto_offset_reset; else - offset = err_offset; + pos.offset = err_pos.offset; - if (offset == RD_KAFKA_OFFSET_INVALID) { + if (pos.offset == RD_KAFKA_OFFSET_INVALID) { /* Error, auto.offset.reset tells us to error out. */ if (broker_id != RD_KAFKA_NODEID_UA) rd_kafka_consumer_err( rktp->rktp_fetchq, broker_id, RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET, 0, NULL, rktp, - err_offset, "%s: %s (broker %" PRId32 ")", reason, - rd_kafka_err2str(err), broker_id); + err_pos.offset, "%s: %s (broker %" PRId32 ")", + reason, rd_kafka_err2str(err), broker_id); else rd_kafka_consumer_err( rktp->rktp_fetchq, broker_id, RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET, 0, NULL, rktp, - err_offset, "%s: %s", reason, + err_pos.offset, "%s: %s", reason, rd_kafka_err2str(err)); rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_NONE); - } else if (offset == RD_KAFKA_OFFSET_BEGINNING && + } else if (pos.offset == RD_KAFKA_OFFSET_BEGINNING && rktp->rktp_lo_offset >= 0) { /* Use cached log start from last Fetch if available. * Note: The cached end offset (rktp_ls_offset) can't be * used here since the End offset is a constantly moving * target as new messages are produced. */ - extra = "cached BEGINNING offset "; - offset = rktp->rktp_lo_offset; - rd_kafka_toppar_next_offset_handle(rktp, offset); + extra = "cached BEGINNING offset "; + pos.offset = rktp->rktp_lo_offset; + pos.leader_epoch = rktp->rktp_leader_epoch; + rd_kafka_toppar_next_offset_handle(rktp, pos); } else { /* Else query cluster for offset */ - rktp->rktp_query_offset = offset; + rktp->rktp_query_pos = pos; rd_kafka_toppar_set_fetch_state( rktp, RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY); } @@ -808,33 +860,30 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, * critical impact. For non-errors, or for auto.offset.reset=error, * the reason is simply debug-logged. */ if (!err || err == RD_KAFKA_RESP_ERR__NO_OFFSET || - offset == RD_KAFKA_OFFSET_INVALID) + pos.offset == RD_KAFKA_OFFSET_INVALID) rd_kafka_dbg( rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", - "%s [%" PRId32 - "]: offset reset (at offset %s, broker %" PRId32 + "%s [%" PRId32 "]: offset reset (at %s, broker %" PRId32 ") " "to %s%s: %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_offset2str(err_offset), broker_id, extra, - rd_kafka_offset2str(offset), reason, rd_kafka_err2str(err)); + rd_kafka_fetch_pos2str(err_pos), broker_id, extra, + rd_kafka_fetch_pos2str(pos), reason, rd_kafka_err2str(err)); else rd_kafka_log( rktp->rktp_rkt->rkt_rk, LOG_WARNING, "OFFSET", - "%s [%" PRId32 - "]: offset reset (at offset %s, broker %" PRId32 - ") " - "to %s%s: %s: %s", + "%s [%" PRId32 "]: offset reset (at %s, broker %" PRId32 + ") to %s%s: %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_offset2str(err_offset), broker_id, extra, - rd_kafka_offset2str(offset), reason, rd_kafka_err2str(err)); + rd_kafka_fetch_pos2str(err_pos), broker_id, extra, + rd_kafka_fetch_pos2str(pos), reason, rd_kafka_err2str(err)); /* Note: If rktp is not delegated to the leader, then low and high offsets will necessarily be cached from the last FETCH request, and so this offset query will never occur in that case for BEGINNING / END logical offsets. */ if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY) - rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_offset, + rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_pos, err ? 100 : 0); } @@ -953,15 +1002,16 @@ static void rd_kafka_offset_file_init(rd_kafka_toppar_t *rktp) { if (offset != RD_KAFKA_OFFSET_INVALID) { /* Start fetching from offset */ - rktp->rktp_stored_offset = offset; - rktp->rktp_committed_offset = offset; - rd_kafka_toppar_next_offset_handle(rktp, offset); + rktp->rktp_stored_pos.offset = offset; + rktp->rktp_committed_pos.offset = offset; + rd_kafka_toppar_next_offset_handle(rktp, rktp->rktp_stored_pos); } else { /* Offset was not usable: perform offset reset logic */ - rktp->rktp_committed_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_committed_pos.offset = RD_KAFKA_OFFSET_INVALID; rd_kafka_offset_reset( - rktp, RD_KAFKA_NODEID_UA, RD_KAFKA_OFFSET_INVALID, + rktp, RD_KAFKA_NODEID_UA, + RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), RD_KAFKA_RESP_ERR__FS, "non-readable offset file"); } } @@ -978,14 +1028,16 @@ rd_kafka_offset_broker_term(rd_kafka_toppar_t *rktp) { /** - * Prepare a toppar for using broker offset commit (broker 0.8.2 or later). - * When using KafkaConsumer (high-level consumer) this functionality is - * disabled in favour of the cgrp commits for the entire set of subscriptions. + * Prepare a toppar for using broker offset commit (broker 0.8.2 or + * later). When using KafkaConsumer (high-level consumer) this + * functionality is disabled in favour of the cgrp commits for the + * entire set of subscriptions. */ static void rd_kafka_offset_broker_init(rd_kafka_toppar_t *rktp) { if (!rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk)) return; - rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, RD_KAFKA_OFFSET_STORED, + rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, + RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_STORED, -1), RD_KAFKA_RESP_ERR_NO_ERROR, "query broker for offsets"); } @@ -1055,23 +1107,27 @@ rd_kafka_resp_err_t rd_kafka_offset_store_stop(rd_kafka_toppar_t *rktp) { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%" PRId32 "]: stopping offset store " - "(stored offset %" PRId64 ", committed offset %" PRId64 - ", EOF offset %" PRId64 ")", + "(stored %s, committed %s, EOF offset %" PRId64 ")", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rktp->rktp_stored_offset, rktp->rktp_committed_offset, + rd_kafka_fetch_pos2str(rktp->rktp_stored_pos), + rd_kafka_fetch_pos2str(rktp->rktp_committed_pos), rktp->rktp_offsets_fin.eof_offset); /* Store end offset for empty partitions */ if (rktp->rktp_rkt->rkt_rk->rk_conf.enable_auto_offset_store && - rktp->rktp_stored_offset == RD_KAFKA_OFFSET_INVALID && + rktp->rktp_stored_pos.offset == RD_KAFKA_OFFSET_INVALID && rktp->rktp_offsets_fin.eof_offset > 0) - rd_kafka_offset_store0(rktp, rktp->rktp_offsets_fin.eof_offset, - rd_true /* force */, RD_DONT_LOCK); + rd_kafka_offset_store0( + rktp, + RD_KAFKA_FETCH_POS(rktp->rktp_offsets_fin.eof_offset, + rktp->rktp_leader_epoch), + rd_true /* force */, RD_DONT_LOCK); /* Commit offset to backing store. * This might be an async operation. */ if (rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) && - rktp->rktp_stored_offset > rktp->rktp_committed_offset) + rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos, + &rktp->rktp_committed_pos) > 0) err = rd_kafka_offset_commit(rktp, "offset store stop"); /* If stop is in progress (async commit), return now. */ @@ -1097,12 +1153,11 @@ void rd_kafka_offset_query_tmr_cb(rd_kafka_timers_t *rkts, void *arg) { rd_kafka_toppar_lock(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "Topic %s [%" PRId32 - "]: timed offset query for %s in " - "state %s", + "]: timed offset query for %s in state %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_offset2str(rktp->rktp_query_offset), + rd_kafka_fetch_pos2str(rktp->rktp_query_pos), rd_kafka_fetch_states[rktp->rktp_fetch_state]); - rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_offset, 0); + rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_pos, 0); rd_kafka_toppar_unlock(rktp); } @@ -1121,7 +1176,7 @@ void rd_kafka_offset_store_init(rd_kafka_toppar_t *rktp) { store_names[rktp->rktp_rkt->rkt_conf.offset_store_method]); /* The committed offset is unknown at this point. */ - rktp->rktp_committed_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_committed_pos.offset = RD_KAFKA_OFFSET_INVALID; /* Set up the commit interval (for simple consumer). */ if (rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) && diff --git a/src/rdkafka_offset.h b/src/rdkafka_offset.h index c085224cb3..6fd9d94407 100644 --- a/src/rdkafka_offset.h +++ b/src/rdkafka_offset.h @@ -71,7 +71,8 @@ const char *rd_kafka_offset2str(int64_t offset); * 4. background rebalance assigns the partition again, but forcibly sets * the stored offset to .._INVALID to provide a clean state. * - * @param offset Offset to set, may be an absolute offset or .._INVALID. + * @param pos Offset and leader epoch to set, may be an absolute offset + * or .._INVALID. * @param force Forcibly set \p offset regardless of assignment state. * @param do_lock Whether to lock the \p rktp or not (already locked by caller). * @@ -82,7 +83,7 @@ const char *rd_kafka_offset2str(int64_t offset); */ static RD_INLINE RD_UNUSED rd_kafka_resp_err_t rd_kafka_offset_store0(rd_kafka_toppar_t *rktp, - int64_t offset, + const rd_kafka_fetch_pos_t pos, rd_bool_t force, rd_dolock_t do_lock) { rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; @@ -90,12 +91,13 @@ rd_kafka_offset_store0(rd_kafka_toppar_t *rktp, if (do_lock) rd_kafka_toppar_lock(rktp); - if (unlikely(!force && !RD_KAFKA_OFFSET_IS_LOGICAL(offset) && + if (unlikely(!force && !RD_KAFKA_OFFSET_IS_LOGICAL(pos.offset) && !(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ASSIGNED) && - !rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk))) + !rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk))) { err = RD_KAFKA_RESP_ERR__STATE; - else - rktp->rktp_stored_offset = offset; + } else { + rktp->rktp_stored_pos = pos; + } if (do_lock) rd_kafka_toppar_unlock(rktp); @@ -115,9 +117,11 @@ void rd_kafka_offset_store_init(rd_kafka_toppar_t *rktp); void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, int32_t broker_id, - int64_t err_offset, + rd_kafka_fetch_pos_t err_pos, rd_kafka_resp_err_t err, - const char *reason); + const char *fmt, + ...) RD_FORMAT(printf, 5, 6); + void rd_kafka_offset_query_tmr_cb(rd_kafka_timers_t *rkts, void *arg); diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 5437eca7f1..19fc5d3643 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -888,7 +888,7 @@ rd_kafka_op_res_t rd_kafka_op_handle(rd_kafka_t *rk, */ void rd_kafka_fetch_op_app_prepare(rd_kafka_t *rk, rd_kafka_op_t *rko) { rd_kafka_toppar_t *rktp; - int64_t offset; + rd_kafka_fetch_pos_t pos; if (unlikely(rko->rko_type != RD_KAFKA_OP_FETCH || rko->rko_err)) return; @@ -898,12 +898,14 @@ void rd_kafka_fetch_op_app_prepare(rd_kafka_t *rk, rd_kafka_op_t *rko) { if (unlikely(!rk)) rk = rktp->rktp_rkt->rkt_rk; - offset = rko->rko_u.fetch.rkm.rkm_rkmessage.offset + 1; + pos.offset = rko->rko_u.fetch.rkm.rkm_rkmessage.offset + 1; + pos.leader_epoch = rko->rko_u.fetch.rkm.rkm_u.consumer.leader_epoch; rd_kafka_toppar_lock(rktp); - rktp->rktp_app_offset = offset; + rktp->rktp_app_pos = pos; + if (rk->rk_conf.enable_auto_offset_store) - rd_kafka_offset_store0(rktp, offset, + rd_kafka_offset_store0(rktp, pos, /* force: ignore assignment state */ rd_true, RD_DONT_LOCK); rd_kafka_toppar_unlock(rktp); diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index db73fe071e..976c91c41b 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -375,13 +375,13 @@ struct rd_kafka_op_s { } node; struct { - int64_t offset; + rd_kafka_fetch_pos_t pos; int32_t broker_id; /**< Originating broker, or -1 */ char *reason; } offset_reset; struct { - int64_t offset; + rd_kafka_fetch_pos_t pos; struct rd_kafka_cgrp_s *rkcg; } fetch_start; /* reused for SEEK */ @@ -745,4 +745,21 @@ void rd_kafka_fetch_op_app_prepare(rd_kafka_t *rk, rd_kafka_op_t *rko); (rd_kafka_replyq_is_valid(&(RKO)->rko_replyq) && \ !rd_kafka_op_version_outdated((RKO), 0)) + + +/** + * @returns the rko for a consumer message (RD_KAFKA_OP_FETCH). + */ +static RD_UNUSED rd_kafka_op_t * +rd_kafka_message2rko(rd_kafka_message_t *rkmessage) { + rd_kafka_op_t *rko = rkmessage->_private; + + if (!rko || rko->rko_type != RD_KAFKA_OP_FETCH) + return NULL; + + return rko; +} + + + #endif /* _RDKAFKA_OP_H_ */ diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 8208944edc..996066a095 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -232,16 +232,16 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, rktp->rktp_offset_fp = NULL; rd_kafka_offset_stats_reset(&rktp->rktp_offsets); rd_kafka_offset_stats_reset(&rktp->rktp_offsets_fin); - rktp->rktp_ls_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_hi_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_lo_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_query_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_next_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_last_next_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_app_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_stored_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_committing_offset = RD_KAFKA_OFFSET_INVALID; - rktp->rktp_committed_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_ls_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_hi_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_lo_offset = RD_KAFKA_OFFSET_INVALID; + rd_kafka_fetch_pos_init(&rktp->rktp_query_pos); + rd_kafka_fetch_pos_init(&rktp->rktp_next_fetch_start); + rd_kafka_fetch_pos_init(&rktp->rktp_last_next_fetch_start); + rd_kafka_fetch_pos_init(&rktp->rktp_app_pos); + rd_kafka_fetch_pos_init(&rktp->rktp_stored_pos); + rd_kafka_fetch_pos_init(&rktp->rktp_committing_pos); + rd_kafka_fetch_pos_init(&rktp->rktp_committed_pos); rd_kafka_msgq_init(&rktp->rktp_msgq); rd_kafka_msgq_init(&rktp->rktp_xmit_msgq); mtx_init(&rktp->rktp_lock, mtx_plain); @@ -368,14 +368,13 @@ void rd_kafka_toppar_set_fetch_state(rd_kafka_toppar_t *rktp, int fetch_state) { rktp->rktp_fetch_state = fetch_state; if (fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE) - rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, - CONSUMER | RD_KAFKA_DBG_TOPIC, "FETCH", - "Partition %.*s [%" PRId32 - "] start fetching " - "at offset %s", - RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, - rd_kafka_offset2str(rktp->rktp_next_offset)); + rd_kafka_dbg( + rktp->rktp_rkt->rkt_rk, CONSUMER | RD_KAFKA_DBG_TOPIC, + "FETCH", + "Partition %.*s [%" PRId32 "] start fetching at %s", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start)); } @@ -1186,7 +1185,8 @@ void rd_kafka_toppar_offset_commit_result( rd_kafka_toppar_lock(rktp); if (!err) - rktp->rktp_committed_offset = offsets->elems[0].offset; + rktp->rktp_committed_pos = + rd_kafka_topic_partition_get_fetch_pos(&offsets->elems[0]); /* When stopping toppars: * Final commit is now done (or failed), propagate. */ @@ -1207,47 +1207,45 @@ void rd_kafka_toppar_offset_commit_result( * Locks: toppar_lock(rktp) must be held */ void rd_kafka_toppar_next_offset_handle(rd_kafka_toppar_t *rktp, - int64_t Offset) { + rd_kafka_fetch_pos_t next_pos) { - if (RD_KAFKA_OFFSET_IS_LOGICAL(Offset)) { + if (RD_KAFKA_OFFSET_IS_LOGICAL(next_pos.offset)) { /* Offset storage returned logical offset (e.g. "end"), * look it up. */ /* Save next offset, even if logical, so that e.g., * assign(BEGINNING) survives a pause+resume, etc. * See issue #2105. */ - rktp->rktp_next_offset = Offset; + rd_kafka_toppar_set_next_fetch_position(rktp, next_pos); - rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, Offset, + rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, next_pos, RD_KAFKA_RESP_ERR_NO_ERROR, "update"); return; } /* Adjust by TAIL count if, if wanted */ - if (rktp->rktp_query_offset <= RD_KAFKA_OFFSET_TAIL_BASE) { - int64_t orig_Offset = Offset; - int64_t tail_cnt = - llabs(rktp->rktp_query_offset - RD_KAFKA_OFFSET_TAIL_BASE); + if (rktp->rktp_query_pos.offset <= RD_KAFKA_OFFSET_TAIL_BASE) { + int64_t orig_offset = next_pos.offset; + int64_t tail_cnt = llabs(rktp->rktp_query_pos.offset - + RD_KAFKA_OFFSET_TAIL_BASE); - if (tail_cnt > Offset) - Offset = 0; + if (tail_cnt > next_pos.offset) + next_pos.offset = 0; else - Offset -= tail_cnt; + next_pos.offset -= tail_cnt; rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "OffsetReply for topic %s [%" PRId32 "]: " "offset %" PRId64 ": adjusting for " - "OFFSET_TAIL(%" PRId64 - "): " - "effective offset %" PRId64, + "OFFSET_TAIL(%" PRId64 "): effective %s", rktp->rktp_rkt->rkt_topic->str, - rktp->rktp_partition, orig_Offset, tail_cnt, - Offset); + rktp->rktp_partition, orig_offset, tail_cnt, + rd_kafka_fetch_pos2str(next_pos)); } - rktp->rktp_next_offset = Offset; + rd_kafka_toppar_set_next_fetch_position(rktp, next_pos); rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); @@ -1310,7 +1308,6 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk, rd_kafka_toppar_t *rktp = opaque; rd_kafka_topic_partition_list_t *offsets; rd_kafka_topic_partition_t *rktpar; - int64_t Offset; int actions = 0; rd_kafka_toppar_lock(rktp); @@ -1343,7 +1340,7 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk, if (!err && !(rktpar = rd_kafka_topic_partition_list_find( offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition))) { - /* Request partition not found in response */ + /* Requested partition not found in response */ err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; actions |= RD_KAFKA_ERR_ACTION_PERMANENT; } @@ -1386,19 +1383,19 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk, * and signal error back to application. */ rd_kafka_offset_reset(rktp, rkb->rkb_nodeid, - rktp->rktp_query_offset, err, + rktp->rktp_query_pos, err, "failed to query logical offset"); rd_kafka_consumer_err( rktp->rktp_fetchq, rkb->rkb_nodeid, err, 0, NULL, rktp, - (rktp->rktp_query_offset <= + (rktp->rktp_query_pos.offset <= RD_KAFKA_OFFSET_TAIL_BASE - ? rktp->rktp_query_offset - + ? rktp->rktp_query_pos.offset - RD_KAFKA_OFFSET_TAIL_BASE - : rktp->rktp_query_offset), + : rktp->rktp_query_pos.offset), "Failed to query logical offset %s: %s", - rd_kafka_offset2str(rktp->rktp_query_offset), + rd_kafka_offset2str(rktp->rktp_query_pos.offset), rd_kafka_err2str(err)); } else { @@ -1408,7 +1405,7 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk, rd_snprintf( tmp, sizeof(tmp), "failed to query logical offset %s: %s", - rd_kafka_offset2str(rktp->rktp_query_offset), + rd_kafka_offset2str(rktp->rktp_query_pos.offset), rd_kafka_err2str(err)); rd_kafka_toppar_offset_retry(rktp, 500, tmp); @@ -1420,21 +1417,27 @@ static void rd_kafka_toppar_handle_Offset(rd_kafka_t *rk, return; } - Offset = rktpar->offset; - rd_kafka_topic_partition_list_destroy(offsets); rd_kafka_toppar_lock(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "Offset %s request for %.*s [%" PRId32 "] " - "returned offset %s (%" PRId64 ")", - rd_kafka_offset2str(rktp->rktp_query_offset), + "returned offset %s (%" PRId64 ") leader epoch %" PRId32, + rd_kafka_offset2str(rktp->rktp_query_pos.offset), RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rd_kafka_offset2str(Offset), Offset); + rktp->rktp_partition, rd_kafka_offset2str(rktpar->offset), + rktpar->offset, + rd_kafka_topic_partition_get_leader_epoch(rktpar)); + - rd_kafka_toppar_next_offset_handle(rktp, Offset); + rd_kafka_toppar_next_offset_handle( + rktp, RD_KAFKA_FETCH_POS( + rktpar->offset, + rd_kafka_topic_partition_get_leader_epoch(rktpar))); rd_kafka_toppar_unlock(rktp); + rd_kafka_topic_partition_list_destroy(offsets); + rd_kafka_toppar_destroy(rktp); /* from request.opaque */ } @@ -1463,12 +1466,12 @@ static void rd_kafka_toppar_offset_retry(rd_kafka_toppar_t *rktp, (tmr_next == -1 || tmr_next > rd_clock() + (backoff_ms * 1000ll)); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", - "%s [%" PRId32 "]: %s: %s for offset %s", + "%s [%" PRId32 "]: %s: %s for %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, reason, restart_tmr ? "(re)starting offset query timer" : "offset query timer already scheduled", - rd_kafka_offset2str(rktp->rktp_query_offset)); + rd_kafka_fetch_pos2str(rktp->rktp_query_pos)); rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY); @@ -1492,7 +1495,7 @@ static void rd_kafka_toppar_offset_retry(rd_kafka_toppar_t *rktp, * Locks: toppar_lock() must be held */ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, - int64_t query_offset, + rd_kafka_fetch_pos_t query_pos, int backoff_ms) { rd_kafka_broker_t *rkb; @@ -1516,7 +1519,7 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, &rktp->rktp_offset_query_tmr, 1 /*lock*/); - if (query_offset == RD_KAFKA_OFFSET_STORED && + if (query_pos.offset == RD_KAFKA_OFFSET_STORED && rktp->rktp_rkt->rkt_conf.offset_store_method == RD_KAFKA_OFFSET_METHOD_BROKER) { /* @@ -1529,6 +1532,7 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, } else { rd_kafka_topic_partition_list_t *offsets; + rd_kafka_topic_partition_t *rktpar; /* * Look up logical offset (end,beginning,tail,..) @@ -1540,19 +1544,19 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, "offset %s (opv %d)", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, - rd_kafka_offset2str(query_offset), + rd_kafka_offset2str(query_pos.offset), rktp->rktp_op_version); rd_kafka_toppar_keep(rktp); /* refcnt for OffsetRequest opaque*/ - if (query_offset <= RD_KAFKA_OFFSET_TAIL_BASE) - query_offset = RD_KAFKA_OFFSET_END; + if (query_pos.offset <= RD_KAFKA_OFFSET_TAIL_BASE) + query_pos.offset = RD_KAFKA_OFFSET_END; offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add( + rktpar = rd_kafka_topic_partition_list_add( offsets, rktp->rktp_rkt->rkt_topic->str, - rktp->rktp_partition) - ->offset = query_offset; + rktp->rktp_partition); + rd_kafka_topic_partition_set_from_fetch_pos(rktpar, query_pos); rd_kafka_ListOffsetsRequest( rkb, offsets, @@ -1574,7 +1578,7 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, * Locks: none */ static void rd_kafka_toppar_fetch_start(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_op_t *rko_orig) { rd_kafka_cgrp_t *rkcg = rko_orig->rko_u.fetch_start.rkcg; rd_kafka_resp_err_t err = 0; @@ -1585,11 +1589,11 @@ static void rd_kafka_toppar_fetch_start(rd_kafka_toppar_t *rktp, rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "FETCH", "Start fetch for %.*s [%" PRId32 "] in " - "state %s at offset %s (v%" PRId32 ")", + "state %s at %s (v%" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rd_kafka_fetch_states[rktp->rktp_fetch_state], - rd_kafka_offset2str(offset), version); + rd_kafka_fetch_pos2str(pos), version); if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_STOPPING) { err = RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS; @@ -1608,22 +1612,23 @@ static void rd_kafka_toppar_fetch_start(rd_kafka_toppar_t *rktp, } - if (offset == RD_KAFKA_OFFSET_BEGINNING || - offset == RD_KAFKA_OFFSET_END || - offset <= RD_KAFKA_OFFSET_TAIL_BASE) { - rd_kafka_toppar_next_offset_handle(rktp, offset); + if (pos.offset == RD_KAFKA_OFFSET_BEGINNING || + pos.offset == RD_KAFKA_OFFSET_END || + pos.offset <= RD_KAFKA_OFFSET_TAIL_BASE) { + rd_kafka_toppar_next_offset_handle(rktp, pos); - } else if (offset == RD_KAFKA_OFFSET_STORED) { + } else if (pos.offset == RD_KAFKA_OFFSET_STORED) { rd_kafka_offset_store_init(rktp); - } else if (offset == RD_KAFKA_OFFSET_INVALID) { - rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, offset, + } else if (pos.offset == RD_KAFKA_OFFSET_INVALID) { + rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, pos, RD_KAFKA_RESP_ERR__NO_OFFSET, "no previously committed offset " "available"); } else { - rktp->rktp_next_offset = offset; + rd_kafka_toppar_set_next_fetch_position(rktp, pos); + rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); @@ -1666,7 +1671,8 @@ void rd_kafka_toppar_fetch_stopped(rd_kafka_toppar_t *rktp, rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_STOPPED); - rktp->rktp_app_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_app_pos.offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_app_pos.leader_epoch = -1; if (rktp->rktp_cgrp) { /* Detach toppar from cgrp */ @@ -1739,7 +1745,7 @@ void rd_kafka_toppar_fetch_stop(rd_kafka_toppar_t *rktp, * Locality: toppar handler thread */ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_op_t *rko_orig) { rd_kafka_resp_err_t err = 0; int32_t version = rko_orig->rko_version; @@ -1747,11 +1753,9 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, rd_kafka_toppar_lock(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "FETCH", - "Seek %.*s [%" PRId32 - "] to offset %s " - "in state %s (v%" PRId32 ")", + "Seek %.*s [%" PRId32 "] to %s in state %s (v%" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rd_kafka_offset2str(offset), + rktp->rktp_partition, rd_kafka_fetch_pos2str(pos), rd_kafka_fetch_states[rktp->rktp_fetch_state], version); @@ -1761,7 +1765,7 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, } else if (!RD_KAFKA_TOPPAR_FETCH_IS_STARTED(rktp->rktp_fetch_state)) { err = RD_KAFKA_RESP_ERR__STATE; goto err_reply; - } else if (offset == RD_KAFKA_OFFSET_STORED) { + } else if (pos.offset == RD_KAFKA_OFFSET_STORED) { err = RD_KAFKA_RESP_ERR__INVALID_ARG; goto err_reply; } @@ -1771,24 +1775,17 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, /* Reset app offsets since seek()ing is analogue to a (re)assign(), * and we want to avoid using the current app offset on resume() * following a seek (#3567). */ - rktp->rktp_app_offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_app_pos.offset = RD_KAFKA_OFFSET_INVALID; + rktp->rktp_app_pos.leader_epoch = -1; /* Abort pending offset lookups. */ if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY) rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers, &rktp->rktp_offset_query_tmr, 1 /*lock*/); - if (RD_KAFKA_OFFSET_IS_LOGICAL(offset)) - rd_kafka_toppar_next_offset_handle(rktp, offset); - else { - rktp->rktp_next_offset = offset; - rd_kafka_toppar_set_fetch_state(rktp, - RD_KAFKA_TOPPAR_FETCH_ACTIVE); - /* Wake-up broker thread which might be idling on IO */ - if (rktp->rktp_broker) - rd_kafka_broker_wakeup(rktp->rktp_broker, "seek done"); - } + rd_kafka_toppar_next_offset_handle(rktp, pos); + /* Signal back to caller thread that seek has commenced, or err */ err_reply: @@ -1799,10 +1796,9 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, rko = rd_kafka_op_new(RD_KAFKA_OP_SEEK | RD_KAFKA_OP_REPLY); - rko->rko_err = err; - rko->rko_u.fetch_start.offset = - rko_orig->rko_u.fetch_start.offset; - rko->rko_rktp = rd_kafka_toppar_keep(rktp); + rko->rko_err = err; + rko->rko_u.fetch_start.pos = rko_orig->rko_u.fetch_start.pos; + rko->rko_rktp = rd_kafka_toppar_keep(rktp); rd_kafka_replyq_enq(&rko_orig->rko_replyq, rko, 0); } @@ -1849,19 +1845,18 @@ static void rd_kafka_toppar_pause_resume(rd_kafka_toppar_t *rktp, if (rk->rk_type == RD_KAFKA_CONSUMER) { /* Save offset of last consumed message+1 as the * next message to fetch on resume. */ - if (rktp->rktp_app_offset != RD_KAFKA_OFFSET_INVALID) { - rktp->rktp_next_offset = rktp->rktp_app_offset; - } + if (rktp->rktp_app_pos.offset != + RD_KAFKA_OFFSET_INVALID) + rd_kafka_toppar_set_next_fetch_position( + rktp, rktp->rktp_app_pos); rd_kafka_dbg( rk, TOPIC, pause ? "PAUSE" : "RESUME", - "%s %s [%" PRId32 - "]: at offset %s " - "(state %s, v%d)", + "%s %s [%" PRId32 "]: at %s (state %s, v%d)", pause ? "Pause" : "Resume", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_offset2str(rktp->rktp_next_offset), + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), rd_kafka_fetch_states[rktp->rktp_fetch_state], version); } else { @@ -1883,16 +1878,14 @@ static void rd_kafka_toppar_pause_resume(rd_kafka_toppar_t *rktp, if (rk->rk_type == RD_KAFKA_CONSUMER) { rd_kafka_dbg( rk, TOPIC, pause ? "PAUSE" : "RESUME", - "%s %s [%" PRId32 - "]: at offset %s " - "(state %s, v%d)", + "%s %s [%" PRId32 "]: at %s (state %s, v%d)", rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE ? "Resuming" : "Not resuming stopped", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_offset2str(rktp->rktp_next_offset), + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), rd_kafka_fetch_states[rktp->rktp_fetch_state], version); @@ -1908,9 +1901,10 @@ static void rd_kafka_toppar_pause_resume(rd_kafka_toppar_t *rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE || rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT) && - rktp->rktp_next_offset == RD_KAFKA_OFFSET_INVALID) + rktp->rktp_next_fetch_start.offset == + RD_KAFKA_OFFSET_INVALID) rd_kafka_toppar_next_offset_handle( - rktp, rktp->rktp_next_offset); + rktp, rktp->rktp_next_fetch_start); } else rd_kafka_dbg( @@ -1996,7 +1990,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, switch ((int)rko->rko_type) { case RD_KAFKA_OP_FETCH_START: - rd_kafka_toppar_fetch_start(rktp, rko->rko_u.fetch_start.offset, + rd_kafka_toppar_fetch_start(rktp, rko->rko_u.fetch_start.pos, rko); break; @@ -2005,7 +1999,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, break; case RD_KAFKA_OP_SEEK: - rd_kafka_toppar_seek(rktp, rko->rko_u.fetch_start.offset, rko); + rd_kafka_toppar_seek(rktp, rko->rko_u.fetch_start.pos, rko); break; case RD_KAFKA_OP_PAUSE: @@ -2023,7 +2017,7 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, /* OffsetFetch reply */ rd_kafka_topic_partition_list_t *offsets = rko->rko_u.offset_fetch.partitions; - int64_t offset = RD_KAFKA_OFFSET_INVALID; + rd_kafka_fetch_pos_t pos = {RD_KAFKA_OFFSET_INVALID, -1}; rktp = rd_kafka_topic_partition_get_toppar( rk, &offsets->elems[0], rd_true /*create-on-miss*/); @@ -2032,7 +2026,8 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, /* Request succeeded but per-partition might have failed */ rko->rko_err = offsets->elems[0].err; - offset = offsets->elems[0].offset; + pos = rd_kafka_topic_partition_get_fetch_pos( + &offsets->elems[0]); } rd_kafka_topic_partition_list_destroy(offsets); @@ -2077,20 +2072,17 @@ static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, } rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", - "%.*s [%" PRId32 - "]: OffsetFetch returned " - "offset %s (%" PRId64 ")", + "%.*s [%" PRId32 "]: OffsetFetch returned %s", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rd_kafka_offset2str(offset), - offset); + rktp->rktp_partition, rd_kafka_fetch_pos2str(pos)); - if (offset > 0) - rktp->rktp_committed_offset = offset; + if (pos.offset > 0) + rktp->rktp_committed_pos = pos; - if (offset >= 0) - rd_kafka_toppar_next_offset_handle(rktp, offset); + if (pos.offset >= 0) + rd_kafka_toppar_next_offset_handle(rktp, pos); else - rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, offset, + rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA, pos, RD_KAFKA_RESP_ERR__NO_OFFSET, "no previously committed offset " "available"); @@ -2135,7 +2127,7 @@ static void rd_kafka_toppar_op0(rd_kafka_toppar_t *rktp, static void rd_kafka_toppar_op(rd_kafka_toppar_t *rktp, rd_kafka_op_type_t type, int32_t version, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_cgrp_t *rkcg, rd_kafka_replyq_t replyq) { rd_kafka_op_t *rko; @@ -2145,7 +2137,7 @@ static void rd_kafka_toppar_op(rd_kafka_toppar_t *rktp, if (type == RD_KAFKA_OP_FETCH_START || type == RD_KAFKA_OP_SEEK) { if (rkcg) rko->rko_u.fetch_start.rkcg = rkcg; - rko->rko_u.fetch_start.offset = offset; + rko->rko_u.fetch_start.pos = pos; } rd_kafka_toppar_op0(rktp, rko, replyq); @@ -2163,7 +2155,7 @@ static void rd_kafka_toppar_op(rd_kafka_toppar_t *rktp, * This is the thread-safe interface that can be called from any thread. */ rd_kafka_resp_err_t rd_kafka_toppar_op_fetch_start(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_q_t *fwdq, rd_kafka_replyq_t replyq) { int32_t version; @@ -2178,14 +2170,12 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_fetch_start(rd_kafka_toppar_t *rktp, version = rd_kafka_toppar_version_new_barrier(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "CONSUMER", - "Start consuming %.*s [%" PRId32 - "] at " - "offset %s (v%" PRId32 ")", + "Start consuming %.*s [%" PRId32 "] at %s (v%" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rd_kafka_offset2str(offset), + rktp->rktp_partition, rd_kafka_fetch_pos2str(pos), version); - rd_kafka_toppar_op(rktp, RD_KAFKA_OP_FETCH_START, version, offset, + rd_kafka_toppar_op(rktp, RD_KAFKA_OP_FETCH_START, version, pos, rktp->rktp_rkt->rkt_rk->rk_cgrp, replyq); return RD_KAFKA_RESP_ERR_NO_ERROR; @@ -2210,22 +2200,24 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_fetch_stop(rd_kafka_toppar_t *rktp, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, version); - rd_kafka_toppar_op(rktp, RD_KAFKA_OP_FETCH_STOP, version, 0, NULL, - replyq); + rd_kafka_toppar_op(rktp, RD_KAFKA_OP_FETCH_STOP, version, + RD_KAFKA_FETCH_POS(-1, -1), NULL, replyq); return RD_KAFKA_RESP_ERR_NO_ERROR; } /** - * Set/Seek offset of a consumed partition (async operation). - * 'offset' is the target offset - * 'replyq' is an optional queue for handling the ack. + * @brief Set/Seek offset of a consumed partition (async operation). + * + * @param offset is the target offset. + * @param leader_epoch is the partition leader epoch, or -1. + * @param replyq is an optional queue for handling the ack. * * This is the thread-safe interface that can be called from any thread. */ rd_kafka_resp_err_t rd_kafka_toppar_op_seek(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_replyq_t replyq) { int32_t version; @@ -2233,15 +2225,12 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_seek(rd_kafka_toppar_t *rktp, version = rd_kafka_toppar_version_new_barrier(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "CONSUMER", - "Seek %.*s [%" PRId32 - "] to " - "offset %s (v%" PRId32 ")", + "Seek %.*s [%" PRId32 "] to %s (v%" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, rd_kafka_offset2str(offset), + rktp->rktp_partition, rd_kafka_fetch_pos2str(pos), version); - rd_kafka_toppar_op(rktp, RD_KAFKA_OP_SEEK, version, offset, NULL, - replyq); + rd_kafka_toppar_op(rktp, RD_KAFKA_OP_SEEK, version, pos, NULL, replyq); return RD_KAFKA_RESP_ERR_NO_ERROR; } @@ -2543,10 +2532,54 @@ rd_kafka_topic_partition_t *rd_kafka_topic_partition_new(const char *topic, return rktpar; } +/** + * @brief Update \p dst with info from \p src. + */ +static void +rd_kafka_topic_partition_update(rd_kafka_topic_partition_t *dst, + const rd_kafka_topic_partition_t *src) { + const rd_kafka_topic_partition_private_t *srcpriv; + rd_kafka_topic_partition_private_t *dstpriv; + + rd_dassert(!strcmp(dst->topic, src->topic)); + rd_dassert(dst->partition == src->partition); + rd_dassert(dst != src); + + dst->offset = src->offset; + dst->opaque = src->opaque; + dst->err = src->err; + + if (src->metadata_size > 0) { + dst->metadata = rd_malloc(src->metadata_size); + dst->metadata_size = src->metadata_size; + ; + memcpy(dst->metadata, src->metadata, dst->metadata_size); + } + + if ((srcpriv = src->_private)) { + dstpriv = rd_kafka_topic_partition_get_private(dst); + if (srcpriv->rktp && !dstpriv->rktp) + dstpriv->rktp = rd_kafka_toppar_keep(srcpriv->rktp); + + rd_assert(dstpriv->rktp == srcpriv->rktp); + + dstpriv->leader_epoch = srcpriv->leader_epoch; + + } else if ((dstpriv = dst->_private)) { + /* No private object in source, reset the leader epoch. */ + dstpriv->leader_epoch = -1; + } +} + rd_kafka_topic_partition_t * rd_kafka_topic_partition_copy(const rd_kafka_topic_partition_t *src) { - return rd_kafka_topic_partition_new(src->topic, src->partition); + rd_kafka_topic_partition_t *dst = + rd_kafka_topic_partition_new(src->topic, src->partition); + + rd_kafka_topic_partition_update(dst, src); + + return dst; } @@ -2607,12 +2640,25 @@ void rd_kafka_topic_partition_set_leader_epoch( int32_t leader_epoch) { rd_kafka_topic_partition_private_t *parpriv; + /* Avoid allocating private_t if clearing the epoch */ + if (leader_epoch == -1 && !rktpar->_private) + return; + parpriv = rd_kafka_topic_partition_get_private(rktpar); parpriv->leader_epoch = leader_epoch; } - +/** + * @brief Set offset and leader epoch from a fetchpos. + */ +void rd_kafka_topic_partition_set_from_fetch_pos( + rd_kafka_topic_partition_t *rktpar, + const rd_kafka_fetch_pos_t fetchpos) { + rktpar->offset = fetchpos.offset; + rd_kafka_topic_partition_set_leader_epoch(rktpar, + fetchpos.leader_epoch); +} /** * @brief Destroy all partitions in list. @@ -2739,43 +2785,6 @@ rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_upsert( } -/** - * @brief Update \p dst with info from \p src. - */ -void rd_kafka_topic_partition_update(rd_kafka_topic_partition_t *dst, - const rd_kafka_topic_partition_t *src) { - const rd_kafka_topic_partition_private_t *srcpriv; - rd_kafka_topic_partition_private_t *dstpriv; - - rd_dassert(!strcmp(dst->topic, src->topic)); - rd_dassert(dst->partition == src->partition); - rd_dassert(dst != src); - - dst->offset = src->offset; - dst->opaque = src->opaque; - dst->err = src->err; - - if (src->metadata_size > 0) { - dst->metadata = rd_malloc(src->metadata_size); - dst->metadata_size = src->metadata_size; - ; - memcpy(dst->metadata, src->metadata, dst->metadata_size); - } - - if ((srcpriv = src->_private)) { - dstpriv = rd_kafka_topic_partition_get_private(dst); - if (srcpriv->rktp && !dstpriv->rktp) - dstpriv->rktp = rd_kafka_toppar_keep(srcpriv->rktp); - - rd_assert(dstpriv->rktp == srcpriv->rktp); - - dstpriv->leader_epoch = srcpriv->leader_epoch; - - } else if ((dstpriv = dst->_private)) { - /* No private object in source, reset the leader epoch. */ - dstpriv->leader_epoch = -1; - } -} /** * @brief Creates a copy of \p rktpar and adds it to \p rktparlist @@ -3130,7 +3139,7 @@ int rd_kafka_topic_partition_list_set_offsets( for (i = 0; i < rktparlist->cnt; i++) { rd_kafka_topic_partition_t *rktpar = &rktparlist->elems[i]; const char *verb = "setting"; - char preamble[80]; + char preamble[128]; *preamble = '\0'; /* Avoid warning */ @@ -3143,15 +3152,18 @@ int rd_kafka_topic_partition_list_set_offsets( if (rk->rk_conf.debug & (RD_KAFKA_DBG_CGRP | RD_KAFKA_DBG_TOPIC)) rd_snprintf(preamble, sizeof(preamble), - "stored offset %" PRId64 - ", committed offset %" PRId64 ": ", - rktp->rktp_stored_offset, - rktp->rktp_committed_offset); - - if (rktp->rktp_stored_offset > - rktp->rktp_committed_offset) { - verb = "setting stored"; - rktpar->offset = rktp->rktp_stored_offset; + "stored %s, committed %s: ", + rd_kafka_fetch_pos2str( + rktp->rktp_stored_pos), + rd_kafka_fetch_pos2str( + rktp->rktp_committed_pos)); + + if (rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos, + &rktp->rktp_committed_pos) > + 0) { + verb = "setting stored"; + rd_kafka_topic_partition_set_from_fetch_pos( + rktpar, rktp->rktp_stored_pos); } else { rktpar->offset = RD_KAFKA_OFFSET_INVALID; } @@ -3160,6 +3172,8 @@ int rd_kafka_topic_partition_list_set_offsets( if (RD_KAFKA_OFFSET_IS_LOGICAL(rktpar->offset)) { verb = "setting default"; rktpar->offset = def_value; + rd_kafka_topic_partition_set_leader_epoch( + rktpar, -1); } else verb = "keeping"; } @@ -3172,13 +3186,15 @@ int rd_kafka_topic_partition_list_set_offsets( rktpar->topic, rktpar->partition, preamble); else - rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_TOPIC, "OFFSET", - "Topic %s [%" PRId32 - "]: " - "%s%s offset %s%s", - rktpar->topic, rktpar->partition, preamble, - verb, rd_kafka_offset2str(rktpar->offset), - is_commit ? " for commit" : ""); + rd_kafka_dbg( + rk, CGRP | RD_KAFKA_DBG_TOPIC, "OFFSET", + "Topic %s [%" PRId32 + "]: " + "%s%s offset %s (leader epoch %" PRId32 ") %s", + rktpar->topic, rktpar->partition, preamble, verb, + rd_kafka_offset2str(rktpar->offset), + rd_kafka_topic_partition_get_leader_epoch(rktpar), + is_commit ? " for commit" : ""); if (!RD_KAFKA_OFFSET_IS_LOGICAL(rktpar->offset)) valid_cnt++; @@ -4214,3 +4230,17 @@ void rd_kafka_partition_leader_destroy_free(void *ptr) { struct rd_kafka_partition_leader *leader = ptr; rd_kafka_partition_leader_destroy(leader); } + + +const char *rd_kafka_fetch_pos2str(const rd_kafka_fetch_pos_t fetchpos) { + static RD_TLS char ret[2][64]; + static int idx; + + idx = (idx + 1) % 2; + + rd_snprintf( + ret[idx], sizeof(ret[idx]), "offset %s (leader epoch %" PRId32 ")", + rd_kafka_offset2str(fetchpos.offset), fetchpos.leader_epoch); + + return ret[idx]; +} diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 8f050a747a..74c252b8e5 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -39,16 +39,17 @@ extern const char *rd_kafka_fetch_states[]; * @brief Offset statistics */ struct offset_stats { - int64_t fetch_offset; /**< Next offset to fetch */ - int64_t eof_offset; /**< Last offset we reported EOF for */ + rd_kafka_fetch_pos_t fetch_pos; /**< Next offset to fetch */ + int64_t eof_offset; /**< Last offset we reported EOF for */ }; /** * @brief Reset offset_stats struct to default values */ static RD_UNUSED void rd_kafka_offset_stats_reset(struct offset_stats *offs) { - offs->fetch_offset = 0; - offs->eof_offset = RD_KAFKA_OFFSET_INVALID; + offs->fetch_pos.offset = 0; + offs->fetch_pos.leader_epoch = -1; + offs->eof_offset = RD_KAFKA_OFFSET_INVALID; } @@ -67,6 +68,48 @@ struct rd_kafka_toppar_err { }; + +/** + * @brief Fetchpos comparator, leader epoch has precedence. + */ +static RD_UNUSED RD_INLINE int +rd_kafka_fetch_pos_cmp(const rd_kafka_fetch_pos_t *a, + const rd_kafka_fetch_pos_t *b) { + if (a->leader_epoch < b->leader_epoch) + return -1; + else if (a->leader_epoch > b->leader_epoch) + return 1; + else if (a->offset < b->offset) + return -1; + else if (a->offset > b->offset) + return 1; + else + return 0; +} + + +static RD_UNUSED RD_INLINE void +rd_kafka_fetch_pos_init(rd_kafka_fetch_pos_t *fetchpos) { + fetchpos->offset = RD_KAFKA_OFFSET_INVALID; + fetchpos->leader_epoch = -1; +} + +const char *rd_kafka_fetch_pos2str(const rd_kafka_fetch_pos_t fetchpos); + +static RD_UNUSED RD_INLINE rd_kafka_fetch_pos_t +rd_kafka_fetch_pos_make(int64_t offset, int32_t leader_epoch) { + rd_kafka_fetch_pos_t fetchpos = {offset, leader_epoch}; + return fetchpos; +} + +#define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ + ({ \ + rd_kafka_fetch_pos_t _fetchpos = {offset, leader_epoch}; \ + _fetchpos; \ + }) + + + typedef TAILQ_HEAD(rd_kafka_toppar_tqhead_s, rd_kafka_toppar_s) rd_kafka_toppar_tqhead_t; @@ -257,25 +300,37 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ * absolute timestamp * expires. */ - int64_t rktp_query_offset; /* Offset to query broker for*/ - int64_t rktp_next_offset; /* Next offset to start - * fetching from. - * Locality: toppar thread */ - int64_t rktp_last_next_offset; /* Last next_offset handled - * by fetch_decide(). - * Locality: broker thread */ - int64_t rktp_app_offset; /* Last offset delivered to - * application + 1. - * Is reset to INVALID_OFFSET - * when partition is - * unassigned/stopped/seeked. */ - int64_t rktp_stored_offset; /* Last stored offset, but - * maybe not committed yet. */ - int64_t rktp_committing_offset; /* Offset currently being - * committed */ - int64_t rktp_committed_offset; /* Last committed offset */ - rd_ts_t rktp_ts_committed_offset; /* Timestamp of last - * commit */ + /** Offset to query broker for. */ + rd_kafka_fetch_pos_t rktp_query_pos; + + /** Next fetch start position. + * This is set up start, seek, resume, etc, to tell + * the fetcher where to start fetching. + * It is not updated for each fetch, see + * rktp_offsets.fetch_pos for that. + * @locality toppar thread */ + rd_kafka_fetch_pos_t rktp_next_fetch_start; + + /** The previous next fetch position. + * @locality toppar thread */ + rd_kafka_fetch_pos_t rktp_last_next_fetch_start; + + /** Application's position. + * This is the latest offset delivered to application + 1. + * It is reset to INVALID_OFFSET when partition is + * unassigned/stopped/seeked. */ + rd_kafka_fetch_pos_t rktp_app_pos; + + /** Last stored offset, but maybe not yet committed. */ + rd_kafka_fetch_pos_t rktp_stored_pos; + + /** Offset currently being committed */ + rd_kafka_fetch_pos_t rktp_committing_pos; + + /** Last (known) committed offset */ + rd_kafka_fetch_pos_t rktp_committed_pos; + + rd_ts_t rktp_ts_committed_offset; /**< Timestamp of last commit */ struct offset_stats rktp_offsets; /* Current offsets. * Locality: broker thread*/ @@ -503,14 +558,14 @@ void rd_kafka_toppar_desired_unlink(rd_kafka_toppar_t *rktp); void rd_kafka_toppar_desired_del(rd_kafka_toppar_t *rktp); void rd_kafka_toppar_next_offset_handle(rd_kafka_toppar_t *rktp, - int64_t Offset); + rd_kafka_fetch_pos_t next_pos); void rd_kafka_toppar_broker_delegate(rd_kafka_toppar_t *rktp, rd_kafka_broker_t *rkb); rd_kafka_resp_err_t rd_kafka_toppar_op_fetch_start(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_q_t *fwdq, rd_kafka_replyq_t replyq); @@ -518,7 +573,7 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_fetch_stop(rd_kafka_toppar_t *rktp, rd_kafka_replyq_t replyq); rd_kafka_resp_err_t rd_kafka_toppar_op_seek(rd_kafka_toppar_t *rktp, - int64_t offset, + rd_kafka_fetch_pos_t pos, rd_kafka_replyq_t replyq); rd_kafka_resp_err_t @@ -537,7 +592,7 @@ void rd_kafka_toppar_offset_fetch(rd_kafka_toppar_t *rktp, rd_kafka_replyq_t replyq); void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, - int64_t query_offset, + rd_kafka_fetch_pos_t query_pos, int backoff_ms); int rd_kafka_toppar_purge_queues(rd_kafka_toppar_t *rktp, @@ -790,6 +845,19 @@ void rd_kafka_topic_partition_list_update( int rd_kafka_topic_partition_leader_cmp(const void *_a, const void *_b); +void rd_kafka_topic_partition_set_from_fetch_pos( + rd_kafka_topic_partition_t *rktpar, + const rd_kafka_fetch_pos_t fetchpos); + +static RD_UNUSED rd_kafka_fetch_pos_t rd_kafka_topic_partition_get_fetch_pos( + const rd_kafka_topic_partition_t *rktpar) { + rd_kafka_fetch_pos_t fetchpos = { + rktpar->offset, rd_kafka_topic_partition_get_leader_epoch(rktpar)}; + + return fetchpos; +} + + /** * @brief Match function that returns true if partition has a valid offset. */ @@ -936,4 +1004,16 @@ static RD_UNUSED int rd_kafka_toppar_topic_cmp(const void *_a, const void *_b) { } +/** + * @brief Set's the partitions next fetch position, i.e., the next offset + * to start fetching from. + * + * @locks_required rd_kafka_toppar_lock(rktp) MUST be held. + */ +static RD_UNUSED RD_INLINE void +rd_kafka_toppar_set_next_fetch_position(rd_kafka_toppar_t *rktp, + rd_kafka_fetch_pos_t next_pos) { + rktp->rktp_next_fetch_start = next_pos; +} + #endif /* _RDKAFKA_PARTITION_H_ */ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 4240affe39..80738a6634 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -753,7 +753,7 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk, int32_t partition; rd_kafka_toppar_t *rktp; rd_kafka_topic_partition_t *rktpar; - int32_t LeaderEpoch; + int32_t LeaderEpoch = -1; int16_t err2; rd_kafka_buf_read_i32(rkbuf, &partition); @@ -788,21 +788,26 @@ rd_kafka_handle_OffsetFetch(rd_kafka_t *rk, rktpar->offset = RD_KAFKA_OFFSET_INVALID; else rktpar->offset = offset; + + rd_kafka_topic_partition_set_leader_epoch(rktpar, + LeaderEpoch); rktpar->err = err2; rd_rkb_dbg(rkb, TOPIC, "OFFSETFETCH", "OffsetFetchResponse: %s [%" PRId32 "] " - "offset %" PRId64 + "offset %" PRId64 ", leader epoch %" PRId32 ", metadata %d byte(s): %s", - topic_name, partition, offset, + topic_name, partition, offset, LeaderEpoch, RD_KAFKAP_STR_LEN(&metadata), rd_kafka_err2name(rktpar->err)); if (update_toppar && !err2 && rktp) { /* Update toppar's committed offset */ rd_kafka_toppar_lock(rktp); - rktp->rktp_committed_offset = rktpar->offset; + rktp->rktp_committed_pos = + rd_kafka_topic_partition_get_fetch_pos( + rktpar); rd_kafka_toppar_unlock(rktp); } @@ -922,8 +927,7 @@ void rd_kafka_op_handle_OffsetFetch(rd_kafka_t *rk, err = rd_kafka_handle_OffsetFetch( rkb->rkb_rk, rkb, err, rkbuf, request, &offsets, rd_false /*dont update rktp*/, rd_false /*dont add part*/, - /* Allow retries if replyq - * is valid */ + /* Allow retries if replyq is valid */ rd_kafka_op_replyq_is_valid(rko)); if (err == RD_KAFKA_RESP_ERR__IN_PROGRESS) { if (offsets) @@ -1302,7 +1306,9 @@ int rd_kafka_OffsetCommitRequest(rd_kafka_broker_t *rkb, /* v6: KIP-101 CommittedLeaderEpoch */ if (ApiVersion >= 6) - rd_kafka_buf_write_i32(rkbuf, -1); + rd_kafka_buf_write_i32( + rkbuf, + rd_kafka_topic_partition_get_leader_epoch(rktpar)); /* v1: TimeStamp */ if (ApiVersion == 1) @@ -1397,8 +1403,7 @@ rd_kafka_OffsetDeleteRequest(rd_kafka_broker_t *rkb, rd_kafka_buf_write_topic_partitions( rkbuf, grpoffsets->partitions, rd_false /*dont skip invalid offsets*/, rd_false /*any offset*/, - rd_false /*dont write offsets*/, rd_false /*dont write epoch*/, - rd_false /*dont write metadata*/); + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -1425,8 +1430,7 @@ rd_kafka_group_MemberState_consumer_write(rd_kafka_buf_t *env_rkbuf, rd_kafka_buf_write_topic_partitions( rkbuf, rkgm->rkgm_assignment, rd_false /*don't skip invalid offsets*/, rd_false /* any offset */, - rd_false /*don't write offsets*/, rd_false /*don't write epoch*/, - rd_false /*don't write metadata*/); + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); rd_kafka_buf_write_kbytes(rkbuf, rkgm->rkgm_userdata); /* Get pointer to binary buffer */ From 34316a88b36cbd0605b497ec4e79b864c88a3f7c Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 21 Dec 2022 16:00:19 +0100 Subject: [PATCH 06/40] rd_kafka_buf_(write|read)_topic_partition_list(): variable field ordering --- src/rdkafka_admin.c | 7 +- src/rdkafka_assignor.c | 5 +- src/rdkafka_cgrp.c | 4 +- src/rdkafka_request.c | 158 ++++++++++++++++++++++------------ src/rdkafka_request.h | 50 +++++++++-- src/rdkafka_sticky_assignor.c | 3 +- src/rdkafka_txnmgr.c | 13 +-- 7 files changed, 162 insertions(+), 78 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index a83477174d..5891cc1fa5 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -3298,7 +3298,9 @@ rd_kafka_DeleteRecordsResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_read_throttle_time(reply); offsets = rd_kafka_buf_read_topic_partitions( - reply, 0, rd_true /*read_offset*/, rd_true /*read_part_errs*/); + reply, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); if (!offsets) rd_kafka_buf_parse_fail(reply, "Failed to parse topic partitions"); @@ -3879,7 +3881,8 @@ rd_kafka_OffsetDeleteResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_read_throttle_time(reply); partitions = rd_kafka_buf_read_topic_partitions( - reply, 16, rd_false /*no offset */, rd_true /*read error*/); + reply, 16, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); if (!partitions) { rd_snprintf(errstr, errstr_size, "Failed to parse OffsetDeleteResponse partitions"); diff --git a/src/rdkafka_assignor.c b/src/rdkafka_assignor.c index dfd1c775f3..c2aa716440 100644 --- a/src/rdkafka_assignor.c +++ b/src/rdkafka_assignor.c @@ -148,9 +148,8 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( rd_kafka_buf_write_topic_partitions( rkbuf, owned_partitions, rd_false /*don't skip invalid offsets*/, - rd_false /*any offset*/, rd_false /*don't write offsets*/, - rd_false /*don't write epoch*/, - rd_false /*don't write metadata*/); + rd_false /*any offset*/, + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); /* Get binary buffer and allocate a new Kafka Bytes with a copy. */ rd_slice_init_full(&rkbuf->rkbuf_reader, &rkbuf->rkbuf_buf); diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index b3f70444ff..a43cc5c1a8 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -1506,7 +1506,7 @@ static void rd_kafka_cgrp_handle_SyncGroup_memberstate( rd_kafka_buf_read_i16(rkbuf, &Version); if (!(assignment = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, rd_false, rd_false))) + rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION))) goto err_parse; rd_kafka_buf_read_bytes(rkbuf, &UserData); @@ -1801,7 +1801,7 @@ static int rd_kafka_group_MemberMetadata_consumer_read( if (Version >= 1 && !(rkgm->rkgm_owned = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, rd_false, rd_false))) + rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION))) goto err; rd_kafka_buf_destroy(rkbuf); diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 80738a6634..4bdcf6ad93 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -201,25 +201,24 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb, * @brief Read a list of topic+partitions+extra from \p rkbuf. * * @param rkbuf buffer to read from - * @param estimated_part_cnt estimated number of partitions to read. - * @param read_part_errs whether or not to read an error per partition. + * @param fields An array of fields to read from the buffer and set on + * the rktpar object, in the specified order, must end + * with RD_KAFKA_TOPIC_PARTITION_FIELD_END. * * @returns a newly allocated list on success, or NULL on parse error. */ -rd_kafka_topic_partition_list_t * -rd_kafka_buf_read_topic_partitions(rd_kafka_buf_t *rkbuf, - size_t estimated_part_cnt, - rd_bool_t read_offset, - rd_bool_t read_part_errs) { +rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions0( + rd_kafka_buf_t *rkbuf, + size_t estimated_part_cnt, + const rd_kafka_topic_partition_field_t *fields) { const int log_decode_errors = LOG_ERR; - int16_t ErrorCode = 0; int32_t TopicArrayCnt; rd_kafka_topic_partition_list_t *parts = NULL; rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX); parts = rd_kafka_topic_partition_list_new( - RD_MAX(TopicArrayCnt, (int)estimated_part_cnt)); + RD_MAX(TopicArrayCnt * 4, (int)estimated_part_cnt)); while (TopicArrayCnt-- > 0) { rd_kafkap_str_t kTopic; @@ -233,24 +232,55 @@ rd_kafka_buf_read_topic_partitions(rd_kafka_buf_t *rkbuf, RD_KAFKAP_STR_DUPA(&topic, &kTopic); while (PartArrayCnt-- > 0) { - int32_t Partition; - int64_t Offset; + int32_t Partition = -1, Epoch = -1234; + int64_t Offset = -1234; + int16_t ErrorCode = 0; rd_kafka_topic_partition_t *rktpar; + int fi; - rd_kafka_buf_read_i32(rkbuf, &Partition); + /* + * Read requested fields + */ + for (fi = 0; + fields[fi] != RD_KAFKA_TOPIC_PARTITION_FIELD_END; + fi++) { + switch (fields[fi]) { + case RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION: + rd_kafka_buf_read_i32(rkbuf, + &Partition); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET: + rd_kafka_buf_read_i64(rkbuf, &Offset); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH: + rd_kafka_buf_read_i32(rkbuf, &Epoch); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_ERR: + rd_kafka_buf_read_i16(rkbuf, + &ErrorCode); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA: + rd_assert(!*"metadata not implemented"); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP: + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_END: + break; + } + } rktpar = rd_kafka_topic_partition_list_add(parts, topic, Partition); - - if (read_offset) { - rd_kafka_buf_read_i64(rkbuf, &Offset); + /* Use dummy sentinel values that are unlikely to be + * seen from the broker to know if we are to set these + * fields or not. */ + if (Offset != -1234) rktpar->offset = Offset; - } + if (Epoch != -1234) + rd_kafka_topic_partition_set_leader_epoch( + rktpar, Epoch); + rktpar->err = ErrorCode; - if (read_part_errs) { - rd_kafka_buf_read_i16(rkbuf, &ErrorCode); - rktpar->err = ErrorCode; - } rd_kafka_buf_skip_tags(rkbuf); } @@ -275,22 +305,18 @@ rd_kafka_buf_read_topic_partitions(rd_kafka_buf_t *rkbuf, * * @remark The \p parts list MUST be sorted. */ -int rd_kafka_buf_write_topic_partitions( +int rd_kafka_buf_write_topic_partitions0( rd_kafka_buf_t *rkbuf, const rd_kafka_topic_partition_list_t *parts, rd_bool_t skip_invalid_offsets, rd_bool_t only_invalid_offsets, - rd_bool_t write_Offset, - rd_bool_t write_Epoch, - rd_bool_t write_Metadata) { + const rd_kafka_topic_partition_field_t *fields) { size_t of_TopicArrayCnt; size_t of_PartArrayCnt = 0; int TopicArrayCnt = 0, PartArrayCnt = 0; int i; const char *prev_topic = NULL; int cnt = 0; - rd_bool_t partition_id_only = - !write_Offset && !write_Epoch && !write_Metadata; rd_assert(!only_invalid_offsets || (only_invalid_offsets != skip_invalid_offsets)); @@ -300,6 +326,7 @@ int rd_kafka_buf_write_topic_partitions( for (i = 0; i < parts->cnt; i++) { const rd_kafka_topic_partition_t *rktpar = &parts->elems[i]; + int fi; if (rktpar->offset < 0) { if (skip_invalid_offsets) @@ -329,36 +356,56 @@ int rd_kafka_buf_write_topic_partitions( rd_kafka_buf_write_arraycnt_pos(rkbuf); } - /* Partition */ - rd_kafka_buf_write_i32(rkbuf, rktpar->partition); - PartArrayCnt++; - /* Time/Offset */ - if (write_Offset) { - rd_kafka_buf_write_i64(rkbuf, rktpar->offset); - } - - if (write_Epoch) { - /* CommittedLeaderEpoch */ - rd_kafka_buf_write_i32(rkbuf, -1); + /* + * Write requested fields + */ + for (fi = 0; fields[fi] != RD_KAFKA_TOPIC_PARTITION_FIELD_END; + fi++) { + switch (fields[fi]) { + case RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION: + rd_kafka_buf_write_i32(rkbuf, + rktpar->partition); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET: + rd_kafka_buf_write_i64(rkbuf, rktpar->offset); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH: + rd_kafka_buf_write_i32( + rkbuf, + rd_kafka_topic_partition_get_leader_epoch( + rktpar)); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_ERR: + rd_kafka_buf_write_i16(rkbuf, rktpar->err); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA: + /* Java client 0.9.0 and broker <0.10.0 can't + * parse Null metadata fields, so as a + * workaround we send an empty string if + * it's Null. */ + if (!rktpar->metadata) + rd_kafka_buf_write_str(rkbuf, "", 0); + else + rd_kafka_buf_write_str( + rkbuf, rktpar->metadata, + rktpar->metadata_size); + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP: + break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_END: + break; + } } - if (write_Metadata) { - /* Metadata */ - /* Java client 0.9.0 and broker <0.10.0 can't parse - * Null metadata fields, so as a workaround we send an - * empty string if it's Null. */ - if (!rktpar->metadata) - rd_kafka_buf_write_str(rkbuf, "", 0); - else - rd_kafka_buf_write_str(rkbuf, rktpar->metadata, - rktpar->metadata_size); - } - /* Tags for partition struct */ - if (!partition_id_only) + if (fi > 1) + /* If there was more than one field written + * then this was a struct and thus needs the + * struct suffix tags written. */ rd_kafka_buf_write_tags(rkbuf); + PartArrayCnt++; cnt++; } @@ -456,7 +503,8 @@ rd_kafka_parse_ListOffsets(rd_kafka_buf_t *rkbuf, int32_t kpartition; int16_t ErrorCode; int32_t OffsetArrayCnt; - int64_t Offset = -1; + int64_t Offset = -1; + int32_t LeaderEpoch = -1; rd_kafka_topic_partition_t *rktpar; rd_kafka_buf_read_i32(rkbuf, &kpartition); @@ -989,8 +1037,8 @@ void rd_kafka_OffsetFetchRequest(rd_kafka_broker_t *rkb, /* Write partition list, filtering out partitions with valid offsets */ PartCnt = rd_kafka_buf_write_topic_partitions( rkbuf, parts, rd_false /*include invalid offsets*/, - rd_false /*skip valid offsets */, rd_false /*don't write offsets*/, - rd_false /*don't write epoch */, rd_false /*don't write metadata*/); + rd_false /*skip valid offsets */, + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); if (ApiVersion >= 7) { /* RequireStable */ @@ -3693,8 +3741,8 @@ rd_kafka_DeleteRecordsRequest(rd_kafka_broker_t *rkb, rd_kafka_buf_write_topic_partitions( rkbuf, partitions, rd_false /*don't skip invalid offsets*/, - rd_false /*any offset*/, rd_true /*do write offsets*/, - rd_false /*don't write epoch*/, rd_false /*don't write metadata*/); + rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET); /* timeout */ op_timeout = rd_kafka_confval_get_int(&options->operation_timeout); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 1c2675d51b..936393d0ea 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -60,19 +60,51 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb, const char *rd_kafka_actions2str(int actions); -rd_kafka_topic_partition_list_t * -rd_kafka_buf_read_topic_partitions(rd_kafka_buf_t *rkbuf, - size_t estimated_part_cnt, - rd_bool_t read_offset, - rd_bool_t read_part_errs); -int rd_kafka_buf_write_topic_partitions( + +typedef enum { + /** Array end sentinel */ + RD_KAFKA_TOPIC_PARTITION_FIELD_END = 0, + /** Read/write int32_t for partition */ + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + /** Read/write int64_t for offset */ + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, + /** Read/write int32_t for leader_epoch */ + RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, + /** Read/write int16_t for error code */ + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + /** Read/write str for metadata */ + RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA, + /** Noop, useful for ternary ifs */ + RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, +} rd_kafka_topic_partition_field_t; + +rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions0( + rd_kafka_buf_t *rkbuf, + size_t estimated_part_cnt, + const rd_kafka_topic_partition_field_t *fields); + +#define rd_kafka_buf_read_topic_partitions(rkbuf, estimated_part_cnt, ...) \ + ({ \ + rd_kafka_topic_partition_field_t _f[] = { \ + __VA_ARGS__, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; \ + rd_kafka_buf_read_topic_partitions0(rkbuf, estimated_part_cnt, \ + _f); \ + }) + +int rd_kafka_buf_write_topic_partitions0( rd_kafka_buf_t *rkbuf, const rd_kafka_topic_partition_list_t *parts, rd_bool_t skip_invalid_offsets, rd_bool_t only_invalid_offsets, - rd_bool_t write_Offset, - rd_bool_t write_Epoch, - rd_bool_t write_Metadata); + const rd_kafka_topic_partition_field_t *fields); +#define rd_kafka_buf_write_topic_partitions(rkbuf, parts, skip_invalid, \ + only_invalid, ...) \ + ({ \ + rd_kafka_topic_partition_field_t _f[] = { \ + __VA_ARGS__, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; \ + rd_kafka_buf_write_topic_partitions0( \ + rkbuf, parts, skip_invalid, only_invalid, _f); \ + }) rd_kafka_resp_err_t rd_kafka_FindCoordinatorRequest(rd_kafka_broker_t *rkb, diff --git a/src/rdkafka_sticky_assignor.c b/src/rdkafka_sticky_assignor.c index d0a6e03167..cac8d17236 100644 --- a/src/rdkafka_sticky_assignor.c +++ b/src/rdkafka_sticky_assignor.c @@ -1866,8 +1866,7 @@ static rd_kafkap_bytes_t *rd_kafka_sticky_assignor_get_metadata( rd_assert(state->prev_assignment != NULL); rd_kafka_buf_write_topic_partitions( rkbuf, state->prev_assignment, rd_false /*skip invalid offsets*/, - rd_false /*any offset*/, rd_false /*write offsets*/, - rd_false /*write epoch*/, rd_false /*write metadata*/); + rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); rd_kafka_buf_write_i32(rkbuf, state->generation_id); /* Get binary buffer and allocate a new Kafka Bytes with a copy. */ diff --git a/src/rdkafka_txnmgr.c b/src/rdkafka_txnmgr.c index 2c69cd7d85..864ca6afed 100644 --- a/src/rdkafka_txnmgr.c +++ b/src/rdkafka_txnmgr.c @@ -1496,8 +1496,9 @@ static void rd_kafka_txn_handle_TxnOffsetCommit(rd_kafka_t *rk, rd_kafka_buf_read_throttle_time(rkbuf); - partitions = - rd_kafka_buf_read_topic_partitions(rkbuf, 0, rd_false, rd_true); + partitions = rd_kafka_buf_read_topic_partitions( + rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); if (!partitions) goto err_parse; @@ -1706,9 +1707,11 @@ rd_kafka_txn_send_TxnOffsetCommitRequest(rd_kafka_broker_t *rkb, /* Write per-partition offsets list */ cnt = rd_kafka_buf_write_topic_partitions( rkbuf, rko->rko_u.txn.offsets, rd_true /*skip invalid offsets*/, - rd_false /*any offset*/, rd_true /*write offsets*/, - ApiVersion >= 2 /*write Epoch (-1) */, rd_true /*write Metadata*/); - + rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, + ApiVersion >= 2 ? RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH + : RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, + RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA); if (!cnt) { /* No valid partition offsets, don't commit. */ rd_kafka_buf_destroy(rkbuf); From b80356d18bd5f80b6d0992c77a8b4e92c5d18dde Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 21 Dec 2022 16:29:22 +0100 Subject: [PATCH 07/40] Added rd_kafka_buf_write_arraycnt() --- src/rdkafka_buf.h | 84 ++++++++++++++++++++++++++++++----------------- 1 file changed, 53 insertions(+), 31 deletions(-) diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h index 0552d89557..f26905ec63 100644 --- a/src/rdkafka_buf.h +++ b/src/rdkafka_buf.h @@ -1072,9 +1072,57 @@ rd_kafka_buf_update_u32(rd_kafka_buf_t *rkbuf, size_t of, uint32_t v) { } +/** + * @brief Write varint-encoded signed value to buffer. + */ +static RD_INLINE size_t rd_kafka_buf_write_varint(rd_kafka_buf_t *rkbuf, + int64_t v) { + char varint[RD_UVARINT_ENC_SIZEOF(v)]; + size_t sz; + + sz = rd_uvarint_enc_i64(varint, sizeof(varint), v); + + return rd_kafka_buf_write(rkbuf, varint, sz); +} + +/** + * @brief Write varint-encoded unsigned value to buffer. + */ +static RD_INLINE size_t rd_kafka_buf_write_uvarint(rd_kafka_buf_t *rkbuf, + uint64_t v) { + char varint[RD_UVARINT_ENC_SIZEOF(v)]; + size_t sz; + + sz = rd_uvarint_enc_u64(varint, sizeof(varint), v); + + return rd_kafka_buf_write(rkbuf, varint, sz); +} + + + +/** + * @brief Write standard or flexver arround count field to buffer. + * Use this when the array count is known beforehand, else use + * rd_kafka_buf_write_arraycnt_pos(). + */ +static RD_INLINE RD_UNUSED size_t +rd_kafka_buf_write_arraycnt(rd_kafka_buf_t *rkbuf, size_t cnt) { + + /* Count must fit in 31-bits minus the per-byte carry-bit */ + rd_assert(cnt + 1 < (size_t)(INT_MAX >> 4)); + + if (!(rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) + return rd_kafka_buf_write_i32(rkbuf, (int32_t)cnt); + + /* CompactArray has a base of 1, 0 is for Null arrays */ + cnt += 1; + return rd_kafka_buf_write_uvarint(rkbuf, (uint64_t)cnt); +} + + /** * @brief Write array count field to buffer (i32) for later update with - * rd_kafka_buf_update_arraycnt(). + * rd_kafka_buf_finalize_arraycnt(). */ #define rd_kafka_buf_write_arraycnt_pos(rkbuf) rd_kafka_buf_write_i32(rkbuf, 0) @@ -1092,11 +1140,11 @@ rd_kafka_buf_update_u32(rd_kafka_buf_t *rkbuf, size_t of, uint32_t v) { * and may thus be costly. */ static RD_INLINE void -rd_kafka_buf_finalize_arraycnt(rd_kafka_buf_t *rkbuf, size_t of, int cnt) { +rd_kafka_buf_finalize_arraycnt(rd_kafka_buf_t *rkbuf, size_t of, size_t cnt) { char buf[sizeof(int32_t)]; size_t sz, r; - rd_assert(cnt >= 0); + rd_assert(cnt < (size_t)INT_MAX); if (!(rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) { rd_kafka_buf_update_i32(rkbuf, of, (int32_t)cnt); @@ -1108,7 +1156,8 @@ rd_kafka_buf_finalize_arraycnt(rd_kafka_buf_t *rkbuf, size_t of, int cnt) { sz = rd_uvarint_enc_u64(buf, sizeof(buf), (uint64_t)cnt); rd_assert(!RD_UVARINT_OVERFLOW(sz)); - + if (cnt < 127) + rd_assert(sz == 1); rd_buf_write_update(&rkbuf->rkbuf_buf, of, buf, sz); if (sz < sizeof(int32_t)) { @@ -1142,33 +1191,6 @@ rd_kafka_buf_update_i64(rd_kafka_buf_t *rkbuf, size_t of, int64_t v) { } -/** - * @brief Write varint-encoded signed value to buffer. - */ -static RD_INLINE size_t rd_kafka_buf_write_varint(rd_kafka_buf_t *rkbuf, - int64_t v) { - char varint[RD_UVARINT_ENC_SIZEOF(v)]; - size_t sz; - - sz = rd_uvarint_enc_i64(varint, sizeof(varint), v); - - return rd_kafka_buf_write(rkbuf, varint, sz); -} - -/** - * @brief Write varint-encoded unsigned value to buffer. - */ -static RD_INLINE size_t rd_kafka_buf_write_uvarint(rd_kafka_buf_t *rkbuf, - uint64_t v) { - char varint[RD_UVARINT_ENC_SIZEOF(v)]; - size_t sz; - - sz = rd_uvarint_enc_u64(varint, sizeof(varint), v); - - return rd_kafka_buf_write(rkbuf, varint, sz); -} - - /** * @brief Write standard (2-byte header) or KIP-482 COMPACT_STRING to buffer. * From f130f66e3f769153910269b93712dda650af0b6a Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 21 Dec 2022 17:09:27 +0100 Subject: [PATCH 08/40] Additional partition leader epoch handling --- INTRODUCTION.md | 2 +- src/rdkafka_fetcher.c | 40 ++++- src/rdkafka_int.h | 21 +-- src/rdkafka_metadata.c | 201 ++++++++++++++++------- src/rdkafka_metadata.h | 5 +- src/rdkafka_metadata_cache.c | 46 +++++- src/rdkafka_mock.c | 26 ++- src/rdkafka_mock_handlers.c | 133 +++++++++++---- src/rdkafka_mock_int.h | 6 + src/rdkafka_msg.c | 13 ++ src/rdkafka_msg.h | 2 + src/rdkafka_offset.c | 304 +++++++++++++++++++++++++++++++++++ src/rdkafka_offset.h | 2 + src/rdkafka_partition.c | 12 +- src/rdkafka_partition.h | 3 + src/rdkafka_request.c | 111 ++++++++++++- src/rdkafka_request.h | 15 ++ src/rdkafka_topic.c | 81 +++++++--- src/rdkafka_topic.h | 17 +- 19 files changed, 904 insertions(+), 136 deletions(-) diff --git a/INTRODUCTION.md b/INTRODUCTION.md index e9fc1fa61c..3ad346b22e 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -1887,7 +1887,7 @@ The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/conf | KIP-289 - Consumer group.id default to NULL | 2.2.0 | Supported | | KIP-294 - SSL endpoint verification | 2.0.0 | Supported | | KIP-302 - Use all addresses for resolved broker hostname | 2.1.0 | Supported | -| KIP-320 - Consumer: handle log truncation | 2.1.0, 2.2.0 | Not supported | +| KIP-320 - Consumer: handle log truncation | 2.1.0, 2.2.0 | Supported | | KIP-322 - DeleteTopics disabled error code | 2.1.0 | Supported | | KIP-339 - AdminAPI: incrementalAlterConfigs | 2.3.0 | Not supported | | KIP-341 - Update Sticky partition assignment data | 2.3.0 | Not supported (superceeded by KIP-429) | diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 5cc40c6c64..b36a20261c 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -171,18 +171,43 @@ static void rd_kafka_fetch_reply_handle_partition_error( rd_kafka_resp_err_t err, int64_t HighwaterMarkOffset) { + rd_rkb_dbg(rkb, FETCH, "FETCHERR", + "%.*s [%" PRId32 "]: Fetch failed at %s: %s", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), + rd_kafka_err2name(err)); + /* Some errors should be passed to the * application while some handled by rdkafka */ switch (err) { /* Errors handled by rdkafka */ + case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: + if (rktp->rktp_broker_id != rktp->rktp_leader_id) + rd_kafka_toppar_delegate_to_leader(rktp); + + rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); + break; + case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR: - case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: - /* Request metadata information update*/ + /* Request metadata information update and retry */ + rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); + break; + + case RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH: + rd_rkb_dbg(rkb, MSG | RD_KAFKA_DBG_CONSUMER, "FETCH", + "Topic %s [%" PRId32 + "]: Fetch failed at %s: %s: broker %" PRId32 + "has not yet caught up on latest metadata: " + "retrying", + rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), + rd_kafka_err2str(err), rktp->rktp_broker_id); rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); break; @@ -658,6 +683,11 @@ rd_kafka_fetch_reply_handle(rd_kafka_broker_t *rkb, +/** + * @broker FetchResponse handling. + * + * @locality broker thread (or any thread if err == __DESTROY). + */ static void rd_kafka_broker_fetch_reply(rd_kafka_t *rk, rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err, @@ -835,9 +865,11 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { /* Partition */ rd_kafka_buf_write_i32(rkbuf, rktp->rktp_partition); - if (rd_kafka_buf_ApiVersion(rkbuf) >= 9) + if (rd_kafka_buf_ApiVersion(rkbuf) >= 9) { /* CurrentLeaderEpoch */ - rd_kafka_buf_write_i32(rkbuf, -1); + rd_kafka_buf_write_i32( + rkbuf, rktp->rktp_offsets.fetch_pos.leader_epoch); + } /* FetchOffset */ rd_kafka_buf_write_i64(rkbuf, diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h index d65978fdb2..74475f04ba 100644 --- a/src/rdkafka_int.h +++ b/src/rdkafka_int.h @@ -82,6 +82,17 @@ struct rd_kafka_toppar_s; typedef struct rd_kafka_lwtopic_s rd_kafka_lwtopic_t; +/** + * Protocol level sanity + */ +#define RD_KAFKAP_BROKERS_MAX 10000 +#define RD_KAFKAP_TOPICS_MAX 1000000 +#define RD_KAFKAP_PARTITIONS_MAX 100000 + + +#define RD_KAFKA_OFFSET_IS_LOGICAL(OFF) ((OFF) < 0) + + /** * @struct Represents a fetch position: * an offset and an partition leader epoch (if known, else -1). @@ -110,16 +121,6 @@ typedef struct rd_kafka_fetch_pos_s { #include "rdkafka_coord.h" #include "rdkafka_mock.h" -/** - * Protocol level sanity - */ -#define RD_KAFKAP_BROKERS_MAX 10000 -#define RD_KAFKAP_TOPICS_MAX 1000000 -#define RD_KAFKAP_PARTITIONS_MAX 100000 - - -#define RD_KAFKA_OFFSET_IS_LOGICAL(OFF) ((OFF) < 0) - /** diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index e647afe5fc..55a640ff26 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -200,6 +200,55 @@ rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { +/** + * @brief Partition (id) comparator for partition_id_leader_epoch struct. + */ +static int rd_kafka_metadata_partition_leader_epoch_cmp(const void *_a, + const void *_b) { + const rd_kafka_partition_leader_epoch_t *a = _a, *b = _b; + return RD_CMP(a->partition_id, b->partition_id); +} + + + +/** + * @brief Update topic state and information based on topic metadata. + * + * @param mdt Topic metadata. + * @param leader_epochs Per-partition leader epoch array, or NULL if not known. + * + * @locality rdkafka main thread + * @locks_acquired rd_kafka_wrlock(rk) + */ +static void rd_kafka_parse_Metadata_update_topic( + rd_kafka_broker_t *rkb, + const rd_kafka_metadata_topic_t *mdt, + const rd_kafka_partition_leader_epoch_t *leader_epochs) { + + rd_rkb_dbg(rkb, METADATA, "METADATA", + /* The indent below is intentional */ + " Topic %s with %i partitions%s%s", mdt->topic, + mdt->partition_cnt, mdt->err ? ": " : "", + mdt->err ? rd_kafka_err2str(mdt->err) : ""); + + /* Ignore metadata completely for temporary errors. (issue #513) + * LEADER_NOT_AVAILABLE: Broker is rebalancing + */ + if (mdt->err == RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE && + mdt->partition_cnt == 0) { + rd_rkb_dbg(rkb, TOPIC, "METADATA", + "Temporary error in metadata reply for " + "topic %s (PartCnt %i): %s: ignoring", + mdt->topic, mdt->partition_cnt, + rd_kafka_err2str(mdt->err)); + } else { + /* Update local topic & partition state based + * on metadata */ + rd_kafka_topic_metadata_update2(rkb, mdt, leader_epochs); + } +} + + /** * @brief Handle a Metadata response message. * @@ -220,7 +269,7 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; - struct rd_kafka_metadata *md; + struct rd_kafka_metadata *md = NULL; size_t rkb_namelen; const int log_decode_errors = LOG_ERR; rd_list_t *missing_topics = NULL; @@ -237,6 +286,18 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; int broker_changes = 0; int cache_changes = 0; + /** This array is reused and resized as necessary to hold per-partition + * leader epochs (ApiVersion >= 7). */ + rd_kafka_partition_leader_epoch_t *leader_epochs = NULL; + /** Number of allocated elements in leader_epochs. */ + size_t leader_epochs_size = 0; + rd_ts_t ts_start = rd_clock(); + + /* Ignore metadata updates when terminating */ + if (rd_kafka_terminating(rkb->rkb_rk)) { + err = RD_KAFKA_RESP_ERR__DESTROY; + goto done; + } rd_kafka_assert(NULL, thrd_is_current(rk->rk_thread)); @@ -346,6 +407,17 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, md->topics[i].topic, md->topics[i].partition_cnt); + /* Resize reused leader_epochs array to fit this partition's + * leader epochs. */ + if (ApiVersion >= 7 && md->topics[i].partition_cnt > 0 && + (size_t)md->topics[i].partition_cnt > leader_epochs_size) { + leader_epochs_size = + RD_MAX(32, md->topics[i].partition_cnt); + leader_epochs = + rd_realloc(leader_epochs, sizeof(*leader_epochs) * + leader_epochs_size); + } + for (j = 0; j < md->topics[i].partition_cnt; j++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].partitions[j].err); @@ -353,6 +425,12 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, md->topics[i].partitions[j].id); rd_kafka_buf_read_i32a( rkbuf, md->topics[i].partitions[j].leader); + if (ApiVersion >= 7) { + leader_epochs[j].partition_id = + md->topics[i].partitions[j].id; + rd_kafka_buf_read_i32( + rkbuf, &leader_epochs[j].leader_epoch); + } /* Replicas */ rd_kafka_buf_read_i32a( @@ -427,23 +505,70 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, k++) rd_kafka_buf_read_i32a( rkbuf, md->topics[i].partitions[j].isrs[k]); + + if (ApiVersion >= 5) { + /* OfflineReplicas int32 array (ignored) */ + int32_t offline_replicas_cnt; + + /* #OfflineReplicas */ + rd_kafka_buf_read_i32(rkbuf, + &offline_replicas_cnt); + rd_kafka_buf_skip(rkbuf, offline_replicas_cnt * + sizeof(int32_t)); + } + } + + /* Ignore topics in blacklist */ + if (rkb->rkb_rk->rk_conf.topic_blacklist && + rd_kafka_pattern_match(rkb->rkb_rk->rk_conf.topic_blacklist, + md->topics[i].topic)) { + rd_rkb_dbg(rkb, TOPIC | RD_KAFKA_DBG_METADATA, + "BLACKLIST", + "Ignoring blacklisted topic \"%s\" " + "in metadata", + md->topics[i].topic); + continue; } + /* Sort partitions by partition id */ qsort(md->topics[i].partitions, md->topics[i].partition_cnt, sizeof(*md->topics[i].partitions), rd_kafka_metadata_partition_id_cmp); + if (leader_epochs_size > 0) { + /* And sort leader_epochs by partition id */ + qsort(leader_epochs, md->topics[i].partition_cnt, + sizeof(*leader_epochs), + rd_kafka_metadata_partition_leader_epoch_cmp); + } + + /* Update topic state based on the topic metadata */ + rd_kafka_parse_Metadata_update_topic(rkb, &md->topics[i], + leader_epochs); + + + if (requested_topics) { + rd_list_free_cb(missing_topics, + rd_list_remove_cmp(missing_topics, + md->topics[i].topic, + (void *)strcmp)); + if (!all_topics) { + /* Only update cache when not asking + * for all topics. */ + + rd_kafka_wrlock(rk); + rd_kafka_metadata_cache_topic_update( + rk, &md->topics[i], + rd_false /*propagate later*/); + cache_changes++; + rd_kafka_wrunlock(rk); + } + } } /* Entire Metadata response now parsed without errors: * update our internal state according to the response. */ - /* Avoid metadata updates when we're terminating. */ - if (rd_kafka_terminating(rkb->rkb_rk)) { - err = RD_KAFKA_RESP_ERR__DESTROY; - goto done; - } - if (md->broker_cnt == 0 && md->topic_cnt == 0) { rd_rkb_dbg(rkb, METADATA, "METADATA", "No brokers or topics in metadata: should retry"); @@ -461,58 +586,6 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, &md->brokers[i], NULL); } - /* Update partition count and leader for each topic we know about */ - for (i = 0; i < md->topic_cnt; i++) { - rd_kafka_metadata_topic_t *mdt = &md->topics[i]; - rd_rkb_dbg(rkb, METADATA, "METADATA", - " Topic #%i/%i: %s with %i partitions%s%s", i, - md->topic_cnt, mdt->topic, mdt->partition_cnt, - mdt->err ? ": " : "", - mdt->err ? rd_kafka_err2str(mdt->err) : ""); - - /* Ignore topics in blacklist */ - if (rkb->rkb_rk->rk_conf.topic_blacklist && - rd_kafka_pattern_match(rkb->rkb_rk->rk_conf.topic_blacklist, - mdt->topic)) { - rd_rkb_dbg(rkb, TOPIC, "BLACKLIST", - "Ignoring blacklisted topic \"%s\" " - "in metadata", - mdt->topic); - continue; - } - - /* Ignore metadata completely for temporary errors. (issue #513) - * LEADER_NOT_AVAILABLE: Broker is rebalancing - */ - if (mdt->err == RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE && - mdt->partition_cnt == 0) { - rd_rkb_dbg(rkb, TOPIC, "METADATA", - "Temporary error in metadata reply for " - "topic %s (PartCnt %i): %s: ignoring", - mdt->topic, mdt->partition_cnt, - rd_kafka_err2str(mdt->err)); - } else { - /* Update local topic & partition state based - * on metadata */ - rd_kafka_topic_metadata_update2(rkb, mdt); - } - - if (requested_topics) { - rd_list_free_cb(missing_topics, - rd_list_remove_cmp(missing_topics, - mdt->topic, - (void *)strcmp)); - if (!all_topics) { - rd_kafka_wrlock(rk); - rd_kafka_metadata_cache_topic_update( - rk, mdt, rd_false /*propagate later*/); - cache_changes++; - rd_kafka_wrunlock(rk); - } - } - } - - /* Requested topics not seen in metadata? Propogate to topic code. */ if (missing_topics) { char *topic; @@ -586,8 +659,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, } if (all_topics) { - rd_kafka_metadata_cache_update(rkb->rkb_rk, md, - 1 /*abs update*/); + /* Expire all cache entries that were not updated. */ + rd_kafka_metadata_cache_evict_by_age(rkb->rkb_rk, ts_start); if (rkb->rkb_rk->rk_full_metadata) rd_kafka_metadata_destroy( @@ -639,6 +712,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, if (missing_topics) rd_list_destroy(missing_topics); + if (leader_epochs) + rd_free(leader_epochs); + /* This metadata request was triggered by someone wanting * the metadata information back as a reply, so send that reply now. * In this case we must not rd_free the metadata memory here, @@ -663,6 +739,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, if (missing_topics) rd_list_destroy(missing_topics); + if (leader_epochs) + rd_free(leader_epochs); + rd_tmpabuf_destroy(&tbuf); return err; diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index b77bc19ed7..5782152006 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -110,8 +110,10 @@ struct rd_kafka_metadata_cache_entry { TAILQ_ENTRY(rd_kafka_metadata_cache_entry) rkmce_link; /* rkmc_expiry */ rd_ts_t rkmce_ts_expires; /* Expire time */ rd_ts_t rkmce_ts_insert; /* Insert time */ + /** Last known leader epochs array (same size as the partition count), + * or NULL if not known. */ rd_kafka_metadata_topic_t rkmce_mtopic; /* Cached topic metadata */ - /* rkmce_partitions memory points here. */ + /* rkmce_topics.partitions memory points here. */ }; @@ -152,6 +154,7 @@ struct rd_kafka_metadata_cache { void rd_kafka_metadata_cache_expiry_start(rd_kafka_t *rk); +int rd_kafka_metadata_cache_evict_by_age(rd_kafka_t *rk, rd_ts_t ts); void rd_kafka_metadata_cache_topic_update(rd_kafka_t *rk, const rd_kafka_metadata_topic_t *mdt, rd_bool_t propagate); diff --git a/src/rdkafka_metadata_cache.c b/src/rdkafka_metadata_cache.c index 822d0cb2f1..514d391a83 100644 --- a/src/rdkafka_metadata_cache.c +++ b/src/rdkafka_metadata_cache.c @@ -126,7 +126,7 @@ static void rd_kafka_metadata_cache_evict_tmr_cb(rd_kafka_timers_t *rkts, * * @returns the number of entries evicted. * - * @locks rd_kafka_wrlock() + * @locks_required rd_kafka_wrlock() */ static int rd_kafka_metadata_cache_evict(rd_kafka_t *rk) { int cnt = 0; @@ -160,6 +160,50 @@ static int rd_kafka_metadata_cache_evict(rd_kafka_t *rk) { } +/** + * @brief Evict timed out entries from cache based on their insert/update time + * rather than expiry time. Any entries older than \p ts will be evicted. + * + * @returns the number of entries evicted. + * + * @locks_required rd_kafka_wrlock() + */ +int rd_kafka_metadata_cache_evict_by_age(rd_kafka_t *rk, rd_ts_t ts) { + int cnt = 0; + struct rd_kafka_metadata_cache_entry *rkmce, *tmp; + + TAILQ_FOREACH_SAFE(rkmce, &rk->rk_metadata_cache.rkmc_expiry, + rkmce_link, tmp) { + if (rkmce->rkmce_ts_insert <= ts) { + rd_kafka_metadata_cache_delete(rk, rkmce, 1); + cnt++; + } + } + + /* Update expiry timer */ + rkmce = TAILQ_FIRST(&rk->rk_metadata_cache.rkmc_expiry); + if (rkmce) + rd_kafka_timer_start(&rk->rk_timers, + &rk->rk_metadata_cache.rkmc_expiry_tmr, + rkmce->rkmce_ts_expires - rd_clock(), + rd_kafka_metadata_cache_evict_tmr_cb, rk); + else + rd_kafka_timer_stop(&rk->rk_timers, + &rk->rk_metadata_cache.rkmc_expiry_tmr, 1); + + rd_kafka_dbg(rk, METADATA, "METADATA", + "Expired %d entries older than %dms from metadata cache " + "(%d entries remain)", + cnt, (int)((rd_clock() - ts) / 1000), + rk->rk_metadata_cache.rkmc_cnt); + + if (cnt) + rd_kafka_metadata_cache_propagate_changes(rk); + + return cnt; +} + + /** * @brief Find cache entry by topic name * diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index e28c66484e..251903cc97 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -365,6 +365,26 @@ static void rd_kafka_mock_partition_set_leader0(rd_kafka_mock_partition_t *mpart, rd_kafka_mock_broker_t *mrkb) { mpart->leader = mrkb; + mpart->leader_epoch++; +} + + +/** + * @brief Verifies that the client-provided leader_epoch matches that of the + * partition, else returns the appropriate error. + */ +rd_kafka_resp_err_t rd_kafka_mock_partition_leader_epoch_check( + const rd_kafka_mock_partition_t *mpart, + int32_t leader_epoch) { + if (likely(leader_epoch == -1 || mpart->leader_epoch == leader_epoch)) + return RD_KAFKA_RESP_ERR_NO_ERROR; + else if (mpart->leader_epoch < leader_epoch) + return RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH; + else if (mpart->leader_epoch > leader_epoch) + return RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH; + + /* NOTREACHED, but avoids warning */ + return RD_KAFKA_RESP_ERR_NO_ERROR; } @@ -494,7 +514,9 @@ static void rd_kafka_mock_partition_init(rd_kafka_mock_topic_t *mtopic, mpart->topic = mtopic; mpart->id = id; - mpart->follower_id = -1; + mpart->follower_id = -1; + mpart->leader_epoch = -1; /* Start at -1 since assign_replicas() will + * bump it right away to 0. */ TAILQ_INIT(&mpart->msgsets); @@ -514,7 +536,7 @@ static void rd_kafka_mock_partition_init(rd_kafka_mock_topic_t *mtopic, rd_kafka_mock_partition_t * rd_kafka_mock_partition_find(const rd_kafka_mock_topic_t *mtopic, int32_t partition) { - if (partition < 0 || partition >= mtopic->partition_cnt) + if (!mtopic || partition < 0 || partition >= mtopic->partition_cnt) return NULL; return (rd_kafka_mock_partition_t *)&mtopic->partitions[partition]; diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 835f4a9731..d648fe1770 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -228,7 +228,8 @@ static int rd_kafka_mock_handle_Fetch(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, PartitionCnt); while (PartitionCnt-- > 0) { - int32_t Partition, CurrentLeaderEpoch, PartMaxBytes; + int32_t Partition, CurrentLeaderEpoch = -1, + PartMaxBytes; int64_t FetchOffset, LogStartOffset; rd_kafka_mock_partition_t *mpart = NULL; rd_kafka_resp_err_t err = all_err; @@ -268,6 +269,11 @@ static int rd_kafka_mock_handle_Fetch(rd_kafka_mock_connection_t *mconn, err = RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION; + if (!err && mpart) + err = + rd_kafka_mock_partition_leader_epoch_check( + mpart, CurrentLeaderEpoch); + /* Find MessageSet for FetchOffset */ if (!err && FetchOffset != mpart->end_offset) { if (on_follower && @@ -280,6 +286,17 @@ static int rd_kafka_mock_handle_Fetch(rd_kafka_mock_connection_t *mconn, on_follower))) err = RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE; + rd_kafka_dbg( + mcluster->rk, MOCK, "MOCK", + "Topic %.*s [%" PRId32 + "] fetch err %s for offset %" PRId64 + " mset %p, on_follower %d, " + "start %" PRId64 ", end_offset %" PRId64 + " epoch %" PRId32, + RD_KAFKAP_STR_PR(&Topic), Partition, + rd_kafka_err2name(err), FetchOffset, mset, + on_follower, mpart->start_offset, + mpart->end_offset, mpart->leader_epoch); } @@ -468,6 +485,10 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn, err = RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION; + if (!err && mpart) + err = + rd_kafka_mock_partition_leader_epoch_check( + mpart, CurrentLeaderEpoch); /* Response: ErrorCode */ rd_kafka_buf_write_i16(resp, err); @@ -500,15 +521,18 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn, if (rkbuf->rkbuf_reqhdr.ApiVersion >= 4) { /* Response: LeaderEpoch */ - rd_kafka_buf_write_i64(resp, -1); + rd_kafka_buf_write_i32( + resp, mpart ? mpart->leader_epoch : -1); } rd_kafka_dbg(mcluster->rk, MOCK, "MOCK", "Topic %.*s [%" PRId32 "] returning " - "offset %" PRId64 " for %s: %s", + "offset %" PRId64 " (leader epoch %" PRId32 + ") for %s: %s", RD_KAFKAP_STR_PR(&Topic), Partition, - Offset, rd_kafka_offset2str(Timestamp), + Offset, mpart ? mpart->leader_epoch : -1, + rd_kafka_offset2str(Timestamp), rd_kafka_err2str(err)); } } @@ -550,13 +574,14 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn, mrkb = rd_kafka_mock_cluster_get_coord(mcluster, RD_KAFKA_COORD_GROUP, &GroupId); if (!mrkb && !all_err) - all_err = RD_KAFKA_RESP_ERR_NOT_COORDINATOR; + all_err = RD_KAFKA_RESP_ERR_NOT_COORDINATOR; // FIXME? check if + // its this mrkb? - rd_kafka_buf_read_i32(rkbuf, &TopicsCnt); + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, 100000); /* Response: #Topics */ - rd_kafka_buf_write_i32(resp, TopicsCnt); + rd_kafka_buf_write_arraycnt(resp, TopicsCnt); while (TopicsCnt-- > 0) { rd_kafkap_str_t Topic; @@ -564,14 +589,14 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_mock_topic_t *mtopic; rd_kafka_buf_read_str(rkbuf, &Topic); - rd_kafka_buf_read_i32(rkbuf, &PartitionCnt); + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt, 100000); mtopic = rd_kafka_mock_topic_find_by_kstr(mcluster, &Topic); /* Response: Topic */ rd_kafka_buf_write_kstr(resp, &Topic); /* Response: #Partitions */ - rd_kafka_buf_write_i32(resp, PartitionCnt); + rd_kafka_buf_write_arraycnt(resp, PartitionCnt); while (PartitionCnt-- > 0) { int32_t Partition; @@ -600,7 +625,8 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn, if (rkbuf->rkbuf_reqhdr.ApiVersion >= 5) { /* Response: CommittedLeaderEpoch */ - rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_i32( + resp, mpart ? mpart->leader_epoch : -1); } /* Response: Metadata */ @@ -610,6 +636,9 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn, /* Response: ErrorCode */ rd_kafka_buf_write_i16(resp, err); + /* Response: Struct tags */ + rd_kafka_buf_write_tags(resp); + if (coff) rd_kafka_dbg(mcluster->rk, MOCK, "MOCK", "Topic %s [%" PRId32 @@ -629,6 +658,12 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn, RD_KAFKAP_STR_PR(&GroupId), rd_kafka_err2str(err)); } + + /* Request: Skip struct tags */ + rd_kafka_buf_skip_tags(rkbuf); + + /* Response: Struct tags */ + rd_kafka_buf_write_tags(resp); } if (rkbuf->rkbuf_reqhdr.ApiVersion >= 2) { @@ -758,6 +793,11 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn, int32_t CommittedLeaderEpoch; rd_kafka_buf_read_i32(rkbuf, &CommittedLeaderEpoch); + + if (!err && mpart) + err = + rd_kafka_mock_partition_leader_epoch_check( + mpart, CommittedLeaderEpoch); } if (rkbuf->rkbuf_reqhdr.ApiVersion == 1) { @@ -837,7 +877,7 @@ rd_kafka_mock_buf_write_Metadata_Topic(rd_kafka_buf_t *resp, if (ApiVersion >= 7) { /* Response: ..Partitions.LeaderEpoch */ - rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_i32(resp, mpart->leader_epoch); } /* Response: ..Partitions.#ReplicaNodes */ @@ -1808,11 +1848,24 @@ rd_kafka_mock_handle_TxnOffsetCommit(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_read_i64(rkbuf, &pid.id); /* Epoch */ rd_kafka_buf_read_i16(rkbuf, &pid.epoch); + + if (rkbuf->rkbuf_reqhdr.ApiVersion >= 3) { + int32_t GenerationId; + rd_kafkap_str_t kMemberId, kGroupInstanceId; + + /* GenerationId */ + rd_kafka_buf_read_i32(rkbuf, &GenerationId); + /* MemberId */ + rd_kafka_buf_read_str(rkbuf, &kMemberId); + /* GroupInstanceId */ + rd_kafka_buf_read_str(rkbuf, &kGroupInstanceId); + } + /* #Topics */ - rd_kafka_buf_read_i32(rkbuf, &TopicsCnt); + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, 100000); /* Response: #Results */ - rd_kafka_buf_write_i32(resp, TopicsCnt); + rd_kafka_buf_write_arraycnt(resp, TopicsCnt); /* Inject error */ err = rd_kafka_mock_next_request_error(mconn, resp); @@ -1828,36 +1881,48 @@ rd_kafka_mock_handle_TxnOffsetCommit(rd_kafka_mock_connection_t *mconn, while (TopicsCnt-- > 0) { rd_kafkap_str_t Topic; int32_t PartsCnt; + rd_kafka_mock_topic_t *mtopic; /* Topic */ rd_kafka_buf_read_str(rkbuf, &Topic); /* Response: Topic */ rd_kafka_buf_write_kstr(resp, &Topic); + mtopic = rd_kafka_mock_topic_find_by_kstr(mcluster, &Topic); + /* #Partitions */ - rd_kafka_buf_read_i32(rkbuf, &PartsCnt); - /* Response: #Partitions */ - rd_kafka_buf_write_i32(resp, PartsCnt); + rd_kafka_buf_read_arraycnt(rkbuf, &PartsCnt, 100000); - /* Ignore if the topic or partition exists or not. */ + /* Response: #Partitions */ + rd_kafka_buf_write_arraycnt(resp, PartsCnt); while (PartsCnt-- > 0) { int32_t Partition; int64_t Offset; rd_kafkap_str_t Metadata; + rd_kafka_mock_partition_t *mpart; /* Partition */ rd_kafka_buf_read_i32(rkbuf, &Partition); /* Response: Partition */ rd_kafka_buf_write_i32(resp, Partition); + mpart = rd_kafka_mock_partition_find(mtopic, Partition); + if (!err && !mpart) + err = RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + /* CommittedOffset */ rd_kafka_buf_read_i64(rkbuf, &Offset); if (rkbuf->rkbuf_reqhdr.ApiVersion >= 2) { /* CommittedLeaderEpoch */ - int32_t Epoch; - rd_kafka_buf_read_i32(rkbuf, &Epoch); + int32_t CommittedLeaderEpoch; + rd_kafka_buf_read_i32(rkbuf, + &CommittedLeaderEpoch); + if (!err && mpart) + err = + rd_kafka_mock_partition_leader_epoch_check( + mpart, CommittedLeaderEpoch); } /* CommittedMetadata */ @@ -1865,7 +1930,19 @@ rd_kafka_mock_handle_TxnOffsetCommit(rd_kafka_mock_connection_t *mconn, /* Response: ErrorCode */ rd_kafka_buf_write_i16(resp, err); + + /* Request: Struct tags */ + rd_kafka_buf_skip_tags(rkbuf); + + /* Response: Struct tags */ + rd_kafka_buf_write_tags(resp); } + + /* Request: Struct tags */ + rd_kafka_buf_skip_tags(rkbuf); + + /* Response: Struct tags */ + rd_kafka_buf_write_tags(resp); } rd_kafka_mock_connection_send_response(mconn, resp); @@ -1940,23 +2017,23 @@ const struct rd_kafka_mock_api_handler [RD_KAFKAP_Produce] = {0, 7, -1, rd_kafka_mock_handle_Produce}, [RD_KAFKAP_Fetch] = {0, 11, -1, rd_kafka_mock_handle_Fetch}, [RD_KAFKAP_ListOffsets] = {0, 5, -1, rd_kafka_mock_handle_ListOffsets}, - [RD_KAFKAP_OffsetFetch] = {0, 5, 6, rd_kafka_mock_handle_OffsetFetch}, - [RD_KAFKAP_OffsetCommit] = {0, 7, 8, rd_kafka_mock_handle_OffsetCommit}, + [RD_KAFKAP_OffsetFetch] = {0, 6, 6, rd_kafka_mock_handle_OffsetFetch}, + [RD_KAFKAP_OffsetCommit] = {0, 8, 8, rd_kafka_mock_handle_OffsetCommit}, [RD_KAFKAP_ApiVersion] = {0, 2, 3, rd_kafka_mock_handle_ApiVersion}, - [RD_KAFKAP_Metadata] = {0, 2, 9, rd_kafka_mock_handle_Metadata}, - [RD_KAFKAP_FindCoordinator] = {0, 2, 3, + [RD_KAFKAP_Metadata] = {0, 9, 9, rd_kafka_mock_handle_Metadata}, + [RD_KAFKAP_FindCoordinator] = {0, 3, 3, rd_kafka_mock_handle_FindCoordinator}, [RD_KAFKAP_InitProducerId] = {0, 4, 2, rd_kafka_mock_handle_InitProducerId}, - [RD_KAFKAP_JoinGroup] = {0, 5, 6, rd_kafka_mock_handle_JoinGroup}, - [RD_KAFKAP_Heartbeat] = {0, 3, 4, rd_kafka_mock_handle_Heartbeat}, - [RD_KAFKAP_LeaveGroup] = {0, 1, 4, rd_kafka_mock_handle_LeaveGroup}, - [RD_KAFKAP_SyncGroup] = {0, 3, 4, rd_kafka_mock_handle_SyncGroup}, + [RD_KAFKAP_JoinGroup] = {0, 6, 6, rd_kafka_mock_handle_JoinGroup}, + [RD_KAFKAP_Heartbeat] = {0, 5, 4, rd_kafka_mock_handle_Heartbeat}, + [RD_KAFKAP_LeaveGroup] = {0, 4, 4, rd_kafka_mock_handle_LeaveGroup}, + [RD_KAFKAP_SyncGroup] = {0, 4, 4, rd_kafka_mock_handle_SyncGroup}, [RD_KAFKAP_AddPartitionsToTxn] = {0, 1, -1, rd_kafka_mock_handle_AddPartitionsToTxn}, [RD_KAFKAP_AddOffsetsToTxn] = {0, 1, -1, rd_kafka_mock_handle_AddOffsetsToTxn}, - [RD_KAFKAP_TxnOffsetCommit] = {0, 2, 3, + [RD_KAFKAP_TxnOffsetCommit] = {0, 3, 3, rd_kafka_mock_handle_TxnOffsetCommit}, [RD_KAFKAP_EndTxn] = {0, 1, -1, rd_kafka_mock_handle_EndTxn}, }; diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 84ccacf02d..be02138606 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -241,6 +241,8 @@ typedef struct rd_kafka_mock_partition_s { TAILQ_ENTRY(rd_kafka_mock_partition_s) leader_link; int32_t id; + int32_t leader_epoch; /**< Leader epoch, bumped on each + * partition leader change. */ int64_t start_offset; /**< Actual/leader start offset */ int64_t end_offset; /**< Actual/leader end offset */ int64_t follower_start_offset; /**< Follower's start offset */ @@ -447,6 +449,10 @@ rd_kafka_mock_partition_log_append(rd_kafka_mock_partition_t *mpart, const rd_kafkap_str_t *TransactionalId, int64_t *BaseOffset); +rd_kafka_resp_err_t rd_kafka_mock_partition_leader_epoch_check( + const rd_kafka_mock_partition_t *mpart, + int32_t leader_epoch); + /** * @returns true if the ApiVersion is supported, else false. diff --git a/src/rdkafka_msg.c b/src/rdkafka_msg.c index ee0e177379..17b67999bb 100644 --- a/src/rdkafka_msg.c +++ b/src/rdkafka_msg.c @@ -1560,6 +1560,19 @@ rd_kafka_message_status(const rd_kafka_message_t *rkmessage) { } +int32_t rd_kafka_message_leader_epoch(const rd_kafka_message_t *rkmessage) { + rd_kafka_msg_t *rkm; + + if (unlikely(!rkmessage->rkt || + rkmessage->rkt->rkt_rk->rk_type != RD_KAFKA_CONSUMER)) + return -1; + + rkm = rd_kafka_message2msg((rd_kafka_message_t *)rkmessage); + + return rkm->rkm_u.consumer.leader_epoch; +} + + void rd_kafka_msgq_dump(FILE *fp, const char *what, rd_kafka_msgq_t *rkmq) { rd_kafka_msg_t *rkm; int cnt = 0; diff --git a/src/rdkafka_msg.h b/src/rdkafka_msg.h index 8546a819e2..877fac15cd 100644 --- a/src/rdkafka_msg.h +++ b/src/rdkafka_msg.h @@ -131,6 +131,8 @@ typedef struct rd_kafka_msg_s { rd_kafkap_bytes_t binhdrs; /**< Unparsed * binary headers in * protocol msg */ + int32_t leader_epoch; /**< Leader epoch at the time + * the message was fetched. */ } consumer; } rkm_u; } rd_kafka_msg_t; diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 615905faf2..c60732e31d 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -52,6 +52,7 @@ #include "rdkafka_partition.h" #include "rdkafka_offset.h" #include "rdkafka_broker.h" +#include "rdkafka_request.h" #include #include @@ -888,6 +889,309 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, } + +/** + * @brief Offset validation retry timer + */ +static void rd_kafka_offset_validate_tmr_cb(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_toppar_t *rktp = arg; + + rd_kafka_toppar_lock(rktp); + rd_kafka_offset_validate(rktp, "retrying offset validation"); + rd_kafka_toppar_unlock(rktp); +} + + + +/** + * @brief OffsetForLeaderEpochResponse handler that + * pushes the matched toppar's to the next state. + * + * @locality rdkafka main thread + */ +static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request, + void *opaque) { + rd_kafka_topic_partition_list_t *parts = NULL; + rd_kafka_toppar_t *rktp = opaque; + rd_kafka_topic_partition_t *rktpar; + int64_t end_offset; + int32_t leader_epoch; + + if (err == RD_KAFKA_RESP_ERR__DESTROY) { + rd_kafka_toppar_destroy(rktp); /* Drop refcnt */ + return; + } + + err = rd_kafka_handle_OffsetForLeaderEpoch(rk, rkb, err, rkbuf, request, + &parts); + + rd_kafka_toppar_lock(rktp); + + if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) + err = RD_KAFKA_RESP_ERR__OUTDATED; + + if (unlikely(!err && parts->cnt == 0)) + err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; + + if (err) { + int actions; + + rd_rkb_dbg(rkb, FETCH, "OFFSETVALID", + "%.*s [%" PRId32 + "]: OffsetForLeaderEpoch requested failed: %s", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, rd_kafka_err2str(err)); + + if (err == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE) { + rd_rkb_dbg(rkb, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: offset and epoch validation not " + "supported by broker: validation skipped", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + /* Reset the epoch to -1 since it can't be used with + * older brokers. */ + rktp->rktp_next_fetch_start.leader_epoch = -1; + rd_kafka_toppar_set_fetch_state( + rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); + goto done; + + } else if (err == RD_KAFKA_RESP_ERR__OUTDATED) { + /* Partition state has changed, this response + * is outdated. */ + goto done; + } + + actions = rd_kafka_err_action( + rkb, err, request, RD_KAFKA_ERR_ACTION_REFRESH, + RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH, + RD_KAFKA_ERR_ACTION_REFRESH, + RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH, + RD_KAFKA_ERR_ACTION_END); + + + if (actions & RD_KAFKA_ERR_ACTION_REFRESH) + rd_kafka_topic_leader_query(rk, rktp->rktp_rkt); + + if (actions & RD_KAFKA_ERR_ACTION_RETRY) { + /* No need for refcnt on rktp for timer opaque + * since the timer resides on the rktp and will be + * stopped on toppar remove. */ + rd_kafka_timer_start_oneshot( + &rk->rk_timers, &rktp->rktp_validate_tmr, rd_false, + 500 * 1000 /* 500ms */, + rd_kafka_offset_validate_tmr_cb, rktp); + goto done; + } + + /* Permanent error */ + rd_kafka_offset_reset( + rktp, rd_kafka_broker_id(rkb), + RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, + rktp->rktp_leader_epoch), + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Unable to validate offset and epoch: %s", + rd_kafka_err2str(err)); + goto done; + } + + + rktpar = &parts->elems[0]; + end_offset = rktpar->offset; + leader_epoch = rd_kafka_topic_partition_get_leader_epoch(rktpar); + + if (end_offset < 0 || leader_epoch < 0) { + rd_kafka_offset_reset(rktp, rd_kafka_broker_id(rkb), + rktp->rktp_next_fetch_start, + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Partition log truncation detected"); + + } else if (end_offset < rktp->rktp_next_fetch_start.offset) { + + if (rktp->rktp_rkt->rkt_conf.auto_offset_reset == + RD_KAFKA_OFFSET_INVALID /* auto.offset.reset=error */) { + rd_kafka_offset_reset( + rktp, rd_kafka_broker_id(rkb), + rktp->rktp_next_fetch_start, + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Partition log truncation detected at %s: " + "broker end offset is %" PRId64 + " (leader epoch %" PRId32 ")", + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), + end_offset, leader_epoch); + + } else { + rd_kafka_error_t *error; + + rd_kafka_toppar_unlock(rktp); + + /* Seek to the updated end offset */ + error = rd_kafka_seek_partitions(rkb->rkb_rk, parts, + RD_POLL_INFINITE); + + if (error) { + rd_kafka_offset_reset( + rktp, rd_kafka_broker_id(rkb), + rktp->rktp_next_fetch_start, + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Partition log truncation detected at %s: " + "broker end offset is %" PRId64 + " (leader epoch %" PRId32 + "): " + "attempted seek to end offset failed: %s", + rd_kafka_fetch_pos2str( + rktp->rktp_next_fetch_start), + end_offset, leader_epoch, + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + } + + rd_kafka_topic_partition_list_destroy(parts); + rd_kafka_toppar_destroy(rktp); + + return; + } + + } else { + rd_rkb_dbg(rkb, FETCH, "OFFSETVALID", + "%.*s [%" PRId32 + "]: offset and epoch validation " + "succeeded: broker end offset %" PRId64 + " (leader epoch %" PRId32 ")", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, end_offset, leader_epoch); + + rktp->rktp_next_fetch_start.leader_epoch = leader_epoch; + rd_kafka_toppar_set_fetch_state(rktp, + RD_KAFKA_TOPPAR_FETCH_ACTIVE); + } + +done: + rd_kafka_toppar_unlock(rktp); + + if (parts) + rd_kafka_topic_partition_list_destroy(parts); + rd_kafka_toppar_destroy(rktp); +} + + +static rd_kafka_op_res_t rd_kafka_offset_validate_op_cb(rd_kafka_t *rk, + rd_kafka_q_t *rkq, + rd_kafka_op_t *rko) { + rd_kafka_toppar_t *rktp = rko->rko_rktp; + rd_kafka_toppar_lock(rktp); + rd_kafka_offset_validate(rktp, "%s", rko->rko_u.offset_reset.reason); + rd_kafka_toppar_unlock(rktp); + return RD_KAFKA_OP_RES_HANDLED; +} + +/** + * @brief Validate partition epoch and offset (KIP-320). + * + * @param rktp the toppar + * @param err Optional error code that triggered the validation. + * @param fmt a reason string for logging. + * + * @locality any. if not main thread, work will be enqued on main thread. + * @locks_required toppar_lock() MUST be held + */ +void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { + rd_kafka_topic_partition_list_t *parts; + rd_kafka_topic_partition_t *rktpar; + char reason[512]; + va_list ap; + + if (rktp->rktp_rkt->rkt_rk->rk_type != RD_KAFKA_CONSUMER) + return; + + va_start(ap, fmt); + rd_vsnprintf(reason, sizeof(reason), fmt, ap); + va_end(ap); + + /* Enqueue op for toppar handler thread if we're on the wrong thread. */ + if (!thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)) { + /* Reuse OP_OFFSET_RESET type */ + rd_kafka_op_t *rko = + rd_kafka_op_new(RD_KAFKA_OP_OFFSET_RESET | RD_KAFKA_OP_CB); + rko->rko_op_cb = rd_kafka_offset_validate_op_cb; + rko->rko_rktp = rd_kafka_toppar_keep(rktp); + rko->rko_u.offset_reset.reason = rd_strdup(reason); + rd_kafka_q_enq(rktp->rktp_ops, rko); + return; + } + + if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE) { + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: skipping offset " + "validation in fetch state %s", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rd_kafka_fetch_states[rktp->rktp_fetch_state]); + return; + } + + /* If the fetch start position does not have an epoch set then + * there is no point in doing validation. + * This is the case for epoch-less seek()s or epoch-less + * committed offsets. */ + if (rktp->rktp_next_fetch_start.leader_epoch == -1) { + rd_kafka_dbg( + rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: skipping offset " + "validation for %s: no leader epoch set", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start)); + return; + } + + if (rktp->rktp_leader_id == -1 || !rktp->rktp_leader || + rktp->rktp_leader->rkb_source == RD_KAFKA_INTERNAL) { + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: unable to perform offset " + "validation: partition leader not available", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + + rd_kafka_toppar_set_fetch_state(rktp, + RD_KAFKA_TOPPAR_FETCH_ACTIVE); + return; + } + + rd_kafka_toppar_set_fetch_state( + rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT); + + /* Construct and send OffsetForLeaderEpochRequest */ + parts = rd_kafka_topic_partition_list_new(1); + rktpar = rd_kafka_topic_partition_list_add( + parts, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); + rd_kafka_topic_partition_set_leader_epoch( + rktpar, rktp->rktp_next_fetch_start.leader_epoch); + rd_kafka_toppar_keep(rktp); /* for request opaque */ + + rd_rkb_dbg(rktp->rktp_leader, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: querying broker for epoch " + "validation of %s: %s", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), reason); + + rd_kafka_OffsetForLeaderEpochRequest( + rktp->rktp_leader, parts, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0), + rd_kafka_toppar_handle_OffsetForLeaderEpoch, rktp); + rd_kafka_topic_partition_list_destroy(parts); +} + + /** * Escape any special characters in filename 'in' and write escaped * string to 'out' (of max size out_size). diff --git a/src/rdkafka_offset.h b/src/rdkafka_offset.h index 6fd9d94407..662acb1347 100644 --- a/src/rdkafka_offset.h +++ b/src/rdkafka_offset.h @@ -122,6 +122,8 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, const char *fmt, ...) RD_FORMAT(printf, 5, 6); +void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) + RD_FORMAT(printf, 2, 3); void rd_kafka_offset_query_tmr_cb(rd_kafka_timers_t *rkts, void *arg); diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 996066a095..897eaa6e98 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -37,8 +37,10 @@ #include "rdunittest.h" -const char *rd_kafka_fetch_states[] = {"none", "stopping", "stopped", - "offset-query", "offset-wait", "active"}; +const char *rd_kafka_fetch_states[] = {"none", "stopping", + "stopped", "offset-query", + "offset-wait", "validate-epoch-wait", + "active"}; static rd_kafka_op_res_t rd_kafka_toppar_op_serve(rd_kafka_t *rk, @@ -301,6 +303,8 @@ static void rd_kafka_toppar_remove(rd_kafka_toppar_t *rktp) { rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp); + rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers, + &rktp->rktp_validate_tmr, 1 /*lock*/); rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers, &rktp->rktp_offset_query_tmr, 1 /*lock*/); rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers, @@ -348,8 +352,8 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp) { /** * Set toppar fetching state. * - * Locality: broker thread - * Locks: rd_kafka_toppar_lock() MUST be held. + * @locality any + * @locks_required rd_kafka_toppar_lock() MUST be held. */ void rd_kafka_toppar_set_fetch_state(rd_kafka_toppar_t *rktp, int fetch_state) { rd_kafka_assert(NULL, diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 74c252b8e5..8ea7b1ae36 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -279,6 +279,7 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ RD_KAFKA_TOPPAR_FETCH_STOPPED, RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, + RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT, RD_KAFKA_TOPPAR_FETCH_ACTIVE, } rktp_fetch_state; /* Broker thread's state */ @@ -416,6 +417,8 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ rd_kafka_timer_t rktp_offset_sync_tmr; /* Offset file sync timer */ rd_kafka_timer_t rktp_consumer_lag_tmr; /* Consumer lag monitoring * timer */ + rd_kafka_timer_t rktp_validate_tmr; /**< Offset and epoch + * validation retry timer */ rd_interval_t rktp_lease_intvl; /**< Preferred replica lease * period */ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 4bdcf6ad93..6bcc0a4b67 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -514,6 +514,9 @@ rd_kafka_parse_ListOffsets(rd_kafka_buf_t *rkbuf, int64_t Timestamp; rd_kafka_buf_read_i64(rkbuf, &Timestamp); rd_kafka_buf_read_i64(rkbuf, &Offset); + if (api_version >= 4) + rd_kafka_buf_read_i32(rkbuf, + &LeaderEpoch); } else if (api_version == 0) { rd_kafka_buf_read_i32(rkbuf, &OffsetArrayCnt); /* We only request one offset so just grab @@ -521,13 +524,15 @@ rd_kafka_parse_ListOffsets(rd_kafka_buf_t *rkbuf, while (OffsetArrayCnt-- > 0) rd_kafka_buf_read_i64(rkbuf, &Offset); } else { - rd_kafka_assert(NULL, !*"NOTREACHED"); + RD_NOTREACHED(); } rktpar = rd_kafka_topic_partition_list_add( offsets, topic_name, kpartition); rktpar->err = ErrorCode; rktpar->offset = Offset; + rd_kafka_topic_partition_set_leader_epoch(rktpar, + LeaderEpoch); if (ErrorCode && !all_err) all_err = ErrorCode; @@ -589,10 +594,14 @@ rd_kafka_handle_ListOffsets(rd_kafka_t *rk, RD_KAFKA_ERR_ACTION_REFRESH | RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH, + RD_KAFKA_ERR_ACTION_REFRESH | RD_KAFKA_ERR_ACTION_RETRY, + RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH, + RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TRANSPORT, RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT, + RD_KAFKA_ERR_ACTION_END); if (actionsp) @@ -637,7 +646,7 @@ rd_kafka_make_ListOffsetsRequest(rd_kafka_broker_t *rkb, int16_t ApiVersion; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_ListOffsets, 0, 2, NULL); + rkb, RD_KAFKAP_ListOffsets, 0, 5, NULL); if (ApiVersion == -1) return RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE; @@ -677,6 +686,12 @@ rd_kafka_make_ListOffsetsRequest(rd_kafka_broker_t *rkb, rd_kafka_buf_write_i32(rkbuf, rktpar->partition); part_cnt++; + if (ApiVersion >= 4) + /* CurrentLeaderEpoch */ + rd_kafka_buf_write_i32( + rkbuf, + rd_kafka_topic_partition_get_leader_epoch(rktpar)); + /* Time/Offset */ rd_kafka_buf_write_i64(rkbuf, rktpar->offset); @@ -736,6 +751,96 @@ void rd_kafka_ListOffsetsRequest(rd_kafka_broker_t *rkb, } +/** + * @brief OffsetForLeaderEpochResponse handler. + */ +rd_kafka_resp_err_t rd_kafka_handle_OffsetForLeaderEpoch( + rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request, + rd_kafka_topic_partition_list_t **offsets) { + const int log_decode_errors = LOG_ERR; + int16_t ApiVersion; + + if (err) + goto err; + + ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion; + + if (ApiVersion >= 2) + rd_kafka_buf_read_throttle_time(rkbuf); + + *offsets = rd_kafka_buf_read_topic_partitions( + rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + ApiVersion >= 1 ? RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH + : RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET); + if (!*offsets) + goto err_parse; + + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err: + return err; + +err_parse: + err = rkbuf->rkbuf_err; + goto err; +} + + +/** + * @brief Send OffsetForLeaderEpochRequest for partition(s). + * + */ +void rd_kafka_OffsetForLeaderEpochRequest( + rd_kafka_broker_t *rkb, + rd_kafka_topic_partition_list_t *parts, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_buf_t *rkbuf; + int16_t ApiVersion; + + ApiVersion = rd_kafka_broker_ApiVersion_supported( + rkb, RD_KAFKAP_OffsetForLeaderEpoch, 2, 2, NULL); + /* If the supported ApiVersions are not yet known, + * or this broker doesn't support it, we let this request + * succeed or fail later from the broker thread where the + * version is checked again. */ + if (ApiVersion == -1) + ApiVersion = 2; + + rkbuf = rd_kafka_buf_new_flexver_request( + rkb, RD_KAFKAP_OffsetForLeaderEpoch, 1, 4 + (parts->cnt * 64), + ApiVersion >= 4 /*flexver*/); + + /* Sort partitions by topic */ + rd_kafka_topic_partition_list_sort_by_topic(parts); + + /* Write partition list */ + rd_kafka_buf_write_topic_partitions( + rkbuf, parts, rd_false /*include invalid offsets*/, + rd_false /*skip valid offsets */, + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + /* CurrentLeaderEpoch */ + RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, + /* LeaderEpoch */ + RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH); + + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); + + /* Let caller perform retries */ + rkbuf->rkbuf_max_retries = RD_KAFKA_REQUEST_NO_RETRIES; + + rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque); +} + + + /** * Generic handler for OffsetFetch responses. * Offsets for included partitions will be propagated through the passed @@ -1954,7 +2059,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, int *full_incr = NULL; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_Metadata, 0, 4, &features); + rkb, RD_KAFKAP_Metadata, 0, 7, &features); rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_Metadata, 1, 4 + (50 * topic_cnt) + 1); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 936393d0ea..0835266efe 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -129,6 +129,21 @@ void rd_kafka_ListOffsetsRequest(rd_kafka_broker_t *rkb, rd_kafka_resp_cb_t *resp_cb, void *opaque); +rd_kafka_resp_err_t +rd_kafka_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request, + rd_kafka_topic_partition_list_t **offsets); +void rd_kafka_OffsetForLeaderEpochRequest( + rd_kafka_broker_t *rkb, + rd_kafka_topic_partition_list_t *parts, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque); + + rd_kafka_resp_err_t rd_kafka_handle_OffsetFetch(rd_kafka_t *rk, rd_kafka_broker_t *rkb, diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 7f79a2ffd5..a15d9cceae 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -34,6 +34,7 @@ #include "rdkafka_broker.h" #include "rdkafka_cgrp.h" #include "rdkafka_metadata.h" +#include "rdkafka_offset.h" #include "rdlog.h" #include "rdsysqueue.h" #include "rdtime.h" @@ -48,10 +49,11 @@ const char *rd_kafka_topic_state_names[] = {"unknown", "exists", "notexists", "error"}; -static int -rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt, - const struct rd_kafka_metadata_topic *mdt, - rd_ts_t ts_insert); +static int rd_kafka_topic_metadata_update( + rd_kafka_topic_t *rkt, + const struct rd_kafka_metadata_topic *mdt, + const rd_kafka_partition_leader_epoch_t *leader_epochs, + rd_ts_t ts_age); /** @@ -476,7 +478,7 @@ rd_kafka_topic_t *rd_kafka_topic_new0(rd_kafka_t *rk, if (existing) *existing = 1; - rd_kafka_topic_metadata_update(rkt, &rkmce->rkmce_mtopic, + rd_kafka_topic_metadata_update(rkt, &rkmce->rkmce_mtopic, NULL, rkmce->rkmce_ts_insert); } @@ -625,6 +627,7 @@ int rd_kafka_toppar_broker_update(rd_kafka_toppar_t *rktp, * @param leader_id The id of the new leader broker. * @param leader A reference to the leader broker or NULL if the * toppar should be undelegated for any reason. + * @param leader_epoch Partition leader's epoch (KIP-320), or -1 if not known. * * @returns 1 if the broker delegation was changed, -1 if the broker * delegation was changed and is now undelegated, else 0. @@ -636,9 +639,10 @@ int rd_kafka_toppar_broker_update(rd_kafka_toppar_t *rktp, static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, int32_t partition, int32_t leader_id, - rd_kafka_broker_t *leader) { + rd_kafka_broker_t *leader, + int32_t leader_epoch) { rd_kafka_toppar_t *rktp; - rd_bool_t fetching_from_follower; + rd_bool_t fetching_from_follower, need_epoch_validation = rd_false; int r = 0; rktp = rd_kafka_toppar_get(rkt, partition, 0); @@ -657,6 +661,32 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, rd_kafka_toppar_lock(rktp); + if (leader_epoch < rktp->rktp_leader_epoch) { + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER", + "%s [%" PRId32 + "]: ignoring outdated metadata update with " + "leader epoch %" PRId32 + " which is older than " + "our cached epoch %" PRId32, + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, leader_epoch, + rktp->rktp_leader_epoch); + rd_kafka_toppar_unlock(rktp); + return 0; + } + + if (leader_epoch > rktp->rktp_leader_epoch) { + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER", + "%s [%" PRId32 "]: leader %" PRId32 + " epoch %" PRId32 " -> leader %" PRId32 + " epoch %" PRId32, + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, rktp->rktp_leader_id, + rktp->rktp_leader_epoch, leader_id, leader_epoch); + rktp->rktp_leader_epoch = leader_epoch; + need_epoch_validation = rd_true; + } + fetching_from_follower = leader != NULL && rktp->rktp_broker != NULL && rktp->rktp_broker->rkb_source != RD_KAFKA_INTERNAL && @@ -690,6 +720,9 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, "leader updated"); } + if (need_epoch_validation) + rd_kafka_offset_validate(rktp, "epoch updated from metadata"); + rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* from get() */ @@ -1187,17 +1220,22 @@ rd_bool_t rd_kafka_topic_set_error(rd_kafka_topic_t *rkt, /** * @brief Update a topic from metadata. * + * @param mdt Topic metadata. + * @param leader_epochs Array of per-partition leader epochs, or NULL. + * The array size is identical to the partition count in + * \p mdt. * @param ts_age absolute age (timestamp) of metadata. * @returns 1 if the number of partitions changed, 0 if not, and -1 if the * topic is unknown. * - * @locks rd_kafka_*lock() MUST be held. + * @locks_required rd_kafka_*lock() MUST be held. */ -static int -rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt, - const struct rd_kafka_metadata_topic *mdt, - rd_ts_t ts_age) { +static int rd_kafka_topic_metadata_update( + rd_kafka_topic_t *rkt, + const struct rd_kafka_metadata_topic *mdt, + const rd_kafka_partition_leader_epoch_t *leader_epochs, + rd_ts_t ts_age) { rd_kafka_t *rk = rkt->rkt_rk; int upd = 0; int j; @@ -1268,11 +1306,14 @@ rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt, for (j = 0; j < mdt->partition_cnt; j++) { int r; rd_kafka_broker_t *leader; + int32_t leader_epoch = + leader_epochs ? leader_epochs[j].leader_epoch : -1; rd_kafka_dbg(rk, TOPIC | RD_KAFKA_DBG_METADATA, "METADATA", - " Topic %s partition %i Leader %" PRId32, + " Topic %s partition %i Leader %" PRId32 + " Epoch %" PRId32, rkt->rkt_topic->str, mdt->partitions[j].id, - mdt->partitions[j].leader); + mdt->partitions[j].leader, leader_epoch); leader = partbrokers[j]; partbrokers[j] = NULL; @@ -1280,7 +1321,7 @@ rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt, /* Update leader for partition */ r = rd_kafka_toppar_leader_update(rkt, mdt->partitions[j].id, mdt->partitions[j].leader, - leader); + leader, leader_epoch); upd += (r != 0 ? 1 : 0); @@ -1336,8 +1377,10 @@ rd_kafka_topic_metadata_update(rd_kafka_topic_t *rkt, * @sa rd_kafka_topic_metadata_update() * @locks none */ -int rd_kafka_topic_metadata_update2(rd_kafka_broker_t *rkb, - const struct rd_kafka_metadata_topic *mdt) { +int rd_kafka_topic_metadata_update2( + rd_kafka_broker_t *rkb, + const struct rd_kafka_metadata_topic *mdt, + const rd_kafka_partition_leader_epoch_t *leader_epochs) { rd_kafka_topic_t *rkt; int r; @@ -1348,7 +1391,7 @@ int rd_kafka_topic_metadata_update2(rd_kafka_broker_t *rkb, return -1; /* Ignore topics that we dont have locally. */ } - r = rd_kafka_topic_metadata_update(rkt, mdt, rd_clock()); + r = rd_kafka_topic_metadata_update(rkt, mdt, leader_epochs, rd_clock()); rd_kafka_wrunlock(rkb->rkb_rk); @@ -1841,6 +1884,6 @@ void rd_ut_kafka_topic_set_topic_exists(rd_kafka_topic_t *rkt, rd_kafka_wrlock(rkt->rkt_rk); rd_kafka_metadata_cache_topic_update(rkt->rkt_rk, &mdt, rd_true); - rd_kafka_topic_metadata_update(rkt, &mdt, rd_clock()); + rd_kafka_topic_metadata_update(rkt, &mdt, NULL, rd_clock()); rd_kafka_wrunlock(rkt->rkt_rk); } diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h index 19e0c02006..29fba60037 100644 --- a/src/rdkafka_topic.h +++ b/src/rdkafka_topic.h @@ -98,6 +98,17 @@ typedef struct rd_kafka_partition_msgid_s { } rd_kafka_partition_msgid_t; +/** + * @struct Aux struct that holds a partition id and a leader epoch. + * Used as temporary holding space for per-partition leader epochs + * while parsing MetadataResponse. + */ +typedef struct rd_kafka_partition_leader_epoch_s { + int32_t partition_id; + int32_t leader_epoch; +} rd_kafka_partition_leader_epoch_t; + + /* * @struct Internal representation of a topic. * @@ -244,8 +255,10 @@ rd_kafka_topic_get_error(rd_kafka_topic_t *rkt) { return err; } -int rd_kafka_topic_metadata_update2(rd_kafka_broker_t *rkb, - const struct rd_kafka_metadata_topic *mdt); +int rd_kafka_topic_metadata_update2( + rd_kafka_broker_t *rkb, + const struct rd_kafka_metadata_topic *mdt, + const rd_kafka_partition_leader_epoch_t *leader_epochs); void rd_kafka_topic_scan_all(rd_kafka_t *rk, rd_ts_t now); From 0b0e94cd79788cba66900ef04b20c003dfef2ebc Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Wed, 21 Dec 2022 17:09:59 +0100 Subject: [PATCH 09/40] Test updates following stricter epoch handling --- tests/0105-transactions_mock.c | 78 ++++++++++++++++++++-------------- 1 file changed, 46 insertions(+), 32 deletions(-) diff --git a/tests/0105-transactions_mock.c b/tests/0105-transactions_mock.c index 0da80661e9..e001188a29 100644 --- a/tests/0105-transactions_mock.c +++ b/tests/0105-transactions_mock.c @@ -180,6 +180,15 @@ create_txn_producer(rd_kafka_mock_cluster_t **mclusterp, if (mclusterp) { *mclusterp = rd_kafka_handle_mock_cluster(rk); TEST_ASSERT(*mclusterp, "failed to create mock cluster"); + + /* Create some of the common consumer "input" topics + * that we must be able to commit to with + * send_offsets_to_transaction(). + * The number depicts the number of partitions in the topic. */ + TEST_CALL_ERR__( + rd_kafka_mock_topic_create(*mclusterp, "srctopic4", 4, 1)); + TEST_CALL_ERR__(rd_kafka_mock_topic_create( + *mclusterp, "srctopic64", 64, 1)); } return rk; @@ -256,11 +265,12 @@ static void do_test_txn_recoverable_errors(void) { * succeed. */ offsets = rd_kafka_topic_partition_list_new(4); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 99)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 39)->offset = 999999111; - rd_kafka_topic_partition_list_add(offsets, "srctopic", 0)->offset = 999; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 3499)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 0)->offset = + 999; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 19)->offset = 123456789; rd_kafka_mock_push_request_errors( @@ -775,11 +785,11 @@ static void do_test_txn_endtxn_errors(void) { * Send some arbitrary offsets. */ offsets = rd_kafka_topic_partition_list_new(4); - rd_kafka_topic_partition_list_add(offsets, "srctopic", + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3) ->offset = 12; - rd_kafka_topic_partition_list_add(offsets, "srctop2", - 99) + rd_kafka_topic_partition_list_add(offsets, "srctopic64", + 60) ->offset = 99999; cgmetadata = @@ -1174,8 +1184,8 @@ static void do_test_txn_req_cnt(void) { * succeed. */ offsets = rd_kafka_topic_partition_list_new(2); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 99)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 40)->offset = 999999111; rd_kafka_mock_push_request_errors(mcluster, RD_KAFKAP_AddOffsetsToTxn, @@ -1247,7 +1257,7 @@ static void do_test_txn_requires_abort_errors(void) { /* Any other transactional API should now raise an error */ offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); @@ -1310,7 +1320,7 @@ static void do_test_txn_requires_abort_errors(void) { RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); error = @@ -1543,8 +1553,8 @@ static void do_test_txn_switch_coordinator_refresh(void) { * Send some arbitrary offsets. */ offsets = rd_kafka_topic_partition_list_new(4); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 99)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 29)->offset = 99999; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); @@ -1671,7 +1681,7 @@ static void do_test_txns_send_offsets_concurrent_is_retried(void) { RD_KAFKA_RESP_ERR_CONCURRENT_TRANSACTIONS); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); @@ -1781,7 +1791,7 @@ static void do_test_txns_no_timeout_crash(void) { /* send_offsets..() should now time out */ offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); error = @@ -1944,11 +1954,12 @@ static void do_test_txn_flush_timeout(void) { * Send some arbitrary offsets. */ offsets = rd_kafka_topic_partition_list_new(4); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 99)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 49)->offset = 999999111; - rd_kafka_topic_partition_list_add(offsets, "srctopic", 0)->offset = 999; - rd_kafka_topic_partition_list_add(offsets, "srctop2", 3499)->offset = + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 0)->offset = + 999; + rd_kafka_topic_partition_list_add(offsets, "srctopic64", 34)->offset = 123456789; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); @@ -2079,7 +2090,7 @@ static void do_test_txn_coord_req_destroy(void) { */ offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3) + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3) ->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); @@ -2262,7 +2273,7 @@ static void do_test_txn_coord_req_multi_find(void) { */ offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new(groupid); @@ -2455,7 +2466,7 @@ static void do_test_unstable_offset_commit(void) { rd_kafka_conf_t *c_conf; rd_kafka_mock_cluster_t *mcluster; rd_kafka_topic_partition_list_t *offsets; - const char *topic = "mytopic"; + const char *topic = "srctopic4"; const int msgcnt = 100; const int64_t offset_to_commit = msgcnt / 2; int i; @@ -2983,7 +2994,7 @@ static void do_test_disconnected_group_coord(rd_bool_t switch_coord) { TEST_SAY("Calling send_offsets_to_transaction()\n"); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, topic, 0)->offset = 1; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 0)->offset = 1; cgmetadata = rd_kafka_consumer_group_metadata_new(grpid); TIMING_START(&timing, "send_offsets_to_transaction(-1)"); @@ -3267,7 +3278,7 @@ static void do_test_txn_resumable_calls_timeout(rd_bool_t do_commit) { mcluster, coord_id, RD_KAFKAP_AddOffsetsToTxn, 1, RD_KAFKA_RESP_ERR_NO_ERROR, 400); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 0)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 0)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); /* This is not a resumable call on timeout */ @@ -3468,8 +3479,8 @@ static int txn_concurrent_thread_main(void *arg) { rd_kafka_consumer_group_metadata_t *cgmetadata = rd_kafka_consumer_group_metadata_new( "mygroupid"); - rd_kafka_topic_partition_list_add(offsets, - "srctopic", 0) + rd_kafka_topic_partition_list_add( + offsets, "srctopic4", 0) ->offset = 12; error = rd_kafka_send_offsets_to_transaction( @@ -3533,11 +3544,14 @@ static void do_test_txn_concurrent_operations(rd_bool_t do_commit) { test_timeout_set(90); - rk = create_txn_producer(&mcluster, transactional_id, 1, NULL); + rk = create_txn_producer(&mcluster, transactional_id, 1, + "socket.connection.setup.timeout.ms", "10000", + NULL); - /* Set broker RTT to 5s so that the background thread has ample - * time to call its conflicting APIs. */ - rd_kafka_mock_broker_set_rtt(mcluster, coord_id, 5000); + /* Set broker RTT to 3.5s so that the background thread has ample + * time to call its conflicting APIs. + * This value must be less than socket.connection.setup.timeout.ms/2. */ + rd_kafka_mock_broker_set_rtt(mcluster, coord_id, 3500); err = rd_kafka_mock_topic_create(mcluster, topic, 1, 1); TEST_ASSERT(!err, "Failed to create topic: %s", rd_kafka_err2str(err)); @@ -3574,7 +3588,7 @@ static void do_test_txn_concurrent_operations(rd_bool_t do_commit) { _start_call("send_offsets_to_transaction"); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 0)->offset = 12; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 0)->offset = 12; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); TEST_CALL_ERROR__( @@ -3767,7 +3781,7 @@ do_test_txn_offset_commit_doesnt_retry_too_quickly(rd_bool_t times_out) { RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE); offsets = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(offsets, "srctopic", 3)->offset = 1; + rd_kafka_topic_partition_list_add(offsets, "srctopic4", 3)->offset = 1; cgmetadata = rd_kafka_consumer_group_metadata_new("mygroupid"); From bb8c5ce8099dbb08a9ce18d3e2bea1f2d3e0c39a Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 2 Jan 2023 10:20:55 +0530 Subject: [PATCH 10/40] WIP: Reset preferred replica after refresh --- src/rdkafka.h | 4 +++- src/rdkafka_fetcher.c | 16 +++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/rdkafka.h b/src/rdkafka.h index ad8ef94305..a8b3f0aef9 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -423,7 +423,9 @@ typedef enum { RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE = 4, /** Leader not available */ RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE = 5, - /** Not leader for partition */ +/** Not leader for partition */ +#define RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER \ + RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION = 6, /** Request timed out */ RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT = 7, diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index b36a20261c..52dbdc2284 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -183,18 +183,16 @@ static void rd_kafka_fetch_reply_handle_partition_error( switch (err) { /* Errors handled by rdkafka */ case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: - if (rktp->rktp_broker_id != rktp->rktp_leader_id) - rd_kafka_toppar_delegate_to_leader(rktp); - - rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); - break; - case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: - case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: - case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: - case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR: + case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: + /* Reset preferred replica before refreshing metadata */ + if (rktp->rktp_broker_id != rktp->rktp_leader_id) + rd_kafka_toppar_delegate_to_leader(rktp); + /* Request metadata information update and retry */ rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); break; From f3b384cb1f284f5ec3f2eb14043711b12c0bb514 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 26 Jan 2023 11:11:45 +0100 Subject: [PATCH 11/40] Add has reliable leader epoch check, upgrading Metadata version to 9 --- INTRODUCTION.md | 2 +- src/rdkafka_metadata.c | 59 ++++++++++++++++++++++++++----------- src/rdkafka_mock_handlers.c | 4 +-- src/rdkafka_request.c | 40 ++++++++++++++++++++----- 4 files changed, 76 insertions(+), 29 deletions(-) diff --git a/INTRODUCTION.md b/INTRODUCTION.md index 1c0f51402f..f542f0c599 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -1953,7 +1953,7 @@ release of librdkafka. | 0 | Produce | 9 | 7 | | 1 | Fetch | 13 | 11 | | 2 | ListOffsets | 7 | 2 | -| 3 | Metadata | 12 | 4 | +| 3 | Metadata | 12 | 9 | | 8 | OffsetCommit | 8 | 7 | | 9 | OffsetFetch | 8 | 7 | | 10 | FindCoordinator | 4 | 2 | diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index 55a640ff26..9967cd6778 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -329,10 +329,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_read_throttle_time(rkbuf); /* Read Brokers */ - rd_kafka_buf_read_i32a(rkbuf, md->broker_cnt); - if (md->broker_cnt > RD_KAFKAP_BROKERS_MAX) - rd_kafka_buf_parse_fail(rkbuf, "Broker_cnt %i > BROKERS_MAX %i", - md->broker_cnt, RD_KAFKAP_BROKERS_MAX); + rd_kafka_buf_read_arraycnt(rkbuf, &md->broker_cnt, + RD_KAFKAP_BROKERS_MAX); if (!(md->brokers = rd_tmpabuf_alloc(&tbuf, md->broker_cnt * sizeof(*md->brokers)))) @@ -350,6 +348,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafkap_str_t rack; rd_kafka_buf_read_str(rkbuf, &rack); } + + rd_kafka_buf_skip_tags(rkbuf); } if (ApiVersion >= 2) @@ -365,15 +365,10 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, /* Read TopicMetadata */ - rd_kafka_buf_read_i32a(rkbuf, md->topic_cnt); + rd_kafka_buf_read_arraycnt(rkbuf, &md->topic_cnt, RD_KAFKAP_TOPICS_MAX); rd_rkb_dbg(rkb, METADATA, "METADATA", "%i brokers, %i topics", md->broker_cnt, md->topic_cnt); - if (md->topic_cnt > RD_KAFKAP_TOPICS_MAX) - rd_kafka_buf_parse_fail( - rkbuf, "TopicMetadata_cnt %" PRId32 " > TOPICS_MAX %i", - md->topic_cnt, RD_KAFKAP_TOPICS_MAX); - if (!(md->topics = rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*md->topics)))) rd_kafka_buf_parse_fail( @@ -389,14 +384,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, } /* PartitionMetadata */ - rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partition_cnt); - if (md->topics[i].partition_cnt > RD_KAFKAP_PARTITIONS_MAX) - rd_kafka_buf_parse_fail(rkbuf, - "TopicMetadata[%i]." - "PartitionMetadata_cnt %i " - "> PARTITIONS_MAX %i", - i, md->topics[i].partition_cnt, - RD_KAFKAP_PARTITIONS_MAX); + rd_kafka_buf_read_arraycnt(rkbuf, &md->topics[i].partition_cnt, + RD_KAFKAP_PARTITIONS_MAX); if (!(md->topics[i].partitions = rd_tmpabuf_alloc( &tbuf, md->topics[i].partition_cnt * @@ -516,8 +505,19 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_skip(rkbuf, offline_replicas_cnt * sizeof(int32_t)); } + + rd_kafka_buf_skip_tags(rkbuf); + } + + if (ApiVersion >= 8) { + int32_t TopicAuthorizedOperations; + /* TopicAuthorizedOperations */ + rd_kafka_buf_read_i32(rkbuf, + &TopicAuthorizedOperations); } + rd_kafka_buf_skip_tags(rkbuf); + /* Ignore topics in blacklist */ if (rkb->rkb_rk->rk_conf.topic_blacklist && rd_kafka_pattern_match(rkb->rkb_rk->rk_conf.topic_blacklist, @@ -530,6 +530,21 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, continue; } + if (leader_epochs_size > 0 && ApiVersion < 9) { + /* Prior to Kafka version 2.4 (which coincides with + * Metadata version 9), the broker does not propagate + * leader epoch information accurately while a + * reassignment is in progress. Relying on a stale + * epoch can lead to FENCED_LEADER_EPOCH errors which + * can prevent consumption throughout the course of + * a reassignment. It is safer in this case to revert + * to the behavior in previous protocol versions + * which checks leader status only. */ + leader_epochs_size = 0; + rd_free(leader_epochs); + leader_epochs = NULL; + } + /* Sort partitions by partition id */ qsort(md->topics[i].partitions, md->topics[i].partition_cnt, @@ -566,6 +581,14 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, } } + if (ApiVersion >= 8 && ApiVersion <= 10) { + int32_t ClusterAuthorizedOperations; + /* ClusterAuthorizedOperations */ + rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); + } + + rd_kafka_buf_skip_tags(rkbuf); + /* Entire Metadata response now parsed without errors: * update our internal state according to the response. */ diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index d648fe1770..32897c611e 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -455,8 +455,8 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, PartitionCnt); while (PartitionCnt-- > 0) { - int32_t Partition, CurrentLeaderEpoch; - int64_t Timestamp, Offset = -1; + int32_t Partition, CurrentLeaderEpoch = -1; + int64_t Timestamp, Offset = -1; int32_t MaxNumOffsets; rd_kafka_mock_partition_t *mpart = NULL; rd_kafka_resp_err_t err = all_err; diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index a67625919a..2f5ff74d6e 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2181,15 +2181,17 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { rd_kafka_buf_t *rkbuf; int16_t ApiVersion = 0; + size_t of_TopicArrayCnt; int features; int topic_cnt = topics ? rd_list_cnt(topics) : 0; int *full_incr = NULL; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_Metadata, 0, 7, &features); + rkb, RD_KAFKAP_Metadata, 0, 8, &features); - rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_Metadata, 1, - 4 + (50 * topic_cnt) + 1); + rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_Metadata, 1, + 4 + (50 * topic_cnt) + 1, + ApiVersion >= 9); if (!reason) reason = ""; @@ -2197,9 +2199,11 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rkbuf->rkbuf_u.Metadata.reason = rd_strdup(reason); rkbuf->rkbuf_u.Metadata.cgrp_update = cgrp_update; + /* TopicArrayCnt */ + of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); + if (!topics && ApiVersion >= 1) { /* a null(0) array (in the protocol) represents no topics */ - rd_kafka_buf_write_i32(rkbuf, 0); rd_rkb_dbg(rkb, METADATA, "METADATA", "Request metadata for brokers only: %s", reason); full_incr = @@ -2210,10 +2214,13 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, full_incr = &rkb->rkb_rk->rk_metadata_cache .rkmc_full_topics_sent; - if (topic_cnt == 0 && ApiVersion >= 1) - rd_kafka_buf_write_i32(rkbuf, -1); /* Null: all topics*/ - else - rd_kafka_buf_write_i32(rkbuf, topic_cnt); + if (topic_cnt == 0 && ApiVersion >= 1 && ApiVersion < 9) { + rd_kafka_buf_update_i32(rkbuf, + of_TopicArrayCnt, -1); /* Null: all topics*/ + } else { + rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, + topic_cnt); + } if (topic_cnt == 0) { rkbuf->rkbuf_u.Metadata.all_topics = 1; @@ -2265,6 +2272,9 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, RD_LIST_FOREACH(topic, topics, i) rd_kafka_buf_write_str(rkbuf, topic, -1); + + /* Tags for previous topic */ + rd_kafka_buf_write_tags(rkbuf); } if (ApiVersion >= 4) { @@ -2286,6 +2296,20 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, "on broker auto.create.topics.enable configuration"); } + if (ApiVersion >= 8 && ApiVersion < 10) { + /* TODO: implement KIP-430 */ + /* IncludeClusterAuthorizedOperations */ + rd_kafka_buf_write_bool(rkbuf, rd_false); + } + + if (ApiVersion >= 8) { + /* TODO: implement KIP-430 */ + /* IncludeTopicAuthorizedOperations */ + rd_kafka_buf_write_bool(rkbuf, rd_false); + } + + /* Tags for the request */ + rd_kafka_buf_write_tags(rkbuf); rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); From 8415d5066c9151f4cabc38de8672905ae815b41f Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 26 Jan 2023 13:02:33 +0100 Subject: [PATCH 12/40] Fix in mock handler: TopicAuthorizedOperations is for each topic --- src/rdkafka_mock_handlers.c | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 32897c611e..f6d29aa8ef 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -896,6 +896,11 @@ rd_kafka_mock_buf_write_Metadata_Topic(rd_kafka_buf_t *resp, rd_kafka_buf_write_i32(resp, 0); } } + + if (ApiVersion >= 8) { + /* Response: Topics.TopicAuthorizedOperations */ + rd_kafka_buf_write_i32(resp, INT32_MIN); + } } @@ -1010,16 +1015,13 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, mtopic, err ? err : mtopic->err); } - if (rkbuf->rkbuf_reqhdr.ApiVersion >= 8) { - /* TopicAuthorizedOperations */ - rd_kafka_buf_write_i32(resp, INT32_MIN); - } } else { /* Response: #Topics: brokers only */ rd_kafka_buf_write_i32(resp, 0); } - if (rkbuf->rkbuf_reqhdr.ApiVersion >= 8) { + if (rkbuf->rkbuf_reqhdr.ApiVersion >= 8 && + rkbuf->rkbuf_reqhdr.ApiVersion <= 10) { /* ClusterAuthorizedOperations */ rd_kafka_buf_write_i32(resp, INT32_MIN); } From 73a371a45c68cb9d81c7033b87cbb6977002cc72 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 27 Jan 2023 14:11:34 +0100 Subject: [PATCH 13/40] Delegate to leader on metadata refresh (#4163) Co-authored-by: Milind L --- src/rdkafka.h | 1 + src/rdkafka_broker.c | 17 ++ src/rdkafka_fetcher.c | 56 +++---- src/rdkafka_request.c | 4 +- tests/0104-fetch_from_follower_mock.c | 232 +++++++++++++++++++++++++- tests/test.c | 89 ++++++++-- tests/test.h | 17 ++ 7 files changed, 371 insertions(+), 45 deletions(-) diff --git a/src/rdkafka.h b/src/rdkafka.h index 764d536854..729158608f 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -6287,6 +6287,7 @@ typedef rd_kafka_resp_err_t(rd_kafka_interceptor_f_on_thread_exit_t)( * @param secproto The security protocol. * @param name The original name of the broker. * @param port The port of the broker. + * @param state Broker state name. * @param ic_opaque The interceptor's opaque pointer specified in ..add..(). * * @returns an error code on failure, the error is logged but otherwise ignored. diff --git a/src/rdkafka_broker.c b/src/rdkafka_broker.c index a32d08d24d..e8fc27b111 100644 --- a/src/rdkafka_broker.c +++ b/src/rdkafka_broker.c @@ -554,6 +554,7 @@ void rd_kafka_broker_fail(rd_kafka_broker_t *rkb, va_list ap; rd_kafka_bufq_t tmpq_waitresp, tmpq; int old_state; + rd_kafka_toppar_t *rktp; rd_kafka_assert(rkb->rkb_rk, thrd_is_current(rkb->rkb_thread)); @@ -641,6 +642,22 @@ void rd_kafka_broker_fail(rd_kafka_broker_t *rkb, rd_kafka_bufq_dump(rkb, "BRKOUTBUFS", &rkb->rkb_outbufs); } + /* If this broker acts as the preferred (follower) replica for any + * partition, delegate the partition back to the leader. */ + TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) { + rd_kafka_toppar_lock(rktp); + if (unlikely(rktp->rktp_broker != rkb)) { + /* Currently migrating away from this + * broker, skip. */ + rd_kafka_toppar_unlock(rktp); + continue; + } + rd_kafka_toppar_unlock(rktp); + + if (rktp->rktp_leader_id != rktp->rktp_broker_id) { + rd_kafka_toppar_delegate_to_leader(rktp); + } + } /* Query for topic leaders to quickly pick up on failover. */ if (err != RD_KAFKA_RESP_ERR__DESTROY && diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 52dbdc2284..4c3cc8a1a9 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -182,33 +182,6 @@ static void rd_kafka_fetch_reply_handle_partition_error( * application while some handled by rdkafka */ switch (err) { /* Errors handled by rdkafka */ - case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: - case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: - case RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER: - case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: - case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR: - case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: - case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: - /* Reset preferred replica before refreshing metadata */ - if (rktp->rktp_broker_id != rktp->rktp_leader_id) - rd_kafka_toppar_delegate_to_leader(rktp); - - /* Request metadata information update and retry */ - rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); - break; - - case RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH: - rd_rkb_dbg(rkb, MSG | RD_KAFKA_DBG_CONSUMER, "FETCH", - "Topic %s [%" PRId32 - "]: Fetch failed at %s: %s: broker %" PRId32 - "has not yet caught up on latest metadata: " - "retrying", - rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), - rd_kafka_err2str(err), rktp->rktp_broker_id); - rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); - break; - case RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE: /* Occurs when: * - Msg exists on broker but @@ -218,8 +191,8 @@ static void rd_kafka_fetch_reply_handle_partition_error( * (replica is out of sync). * - partition leader is out of sync. * - * Handle by requesting metadata update and - * retrying FETCH (with backoff). + * Handle by requesting metadata update, changing back to the + * leader, and then retrying FETCH (with backoff). */ rd_rkb_dbg(rkb, MSG, "FETCH", "Topic %s [%" PRId32 @@ -229,7 +202,30 @@ static void rd_kafka_fetch_reply_handle_partition_error( rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), rktp->rktp_broker_id, rktp->rktp_leader_id); - rd_kafka_topic_fast_leader_query(rkb->rkb_rk); + /* Continue with next case */ + case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: + case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER: + case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR: + case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: + if (rktp->rktp_broker_id != rktp->rktp_leader_id) { + rd_kafka_toppar_delegate_to_leader(rktp); + } + /* Request metadata information update*/ + rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); + break; + + case RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH: + rd_rkb_dbg(rkb, MSG | RD_KAFKA_DBG_CONSUMER, "FETCH", + "Topic %s [%" PRId32 + "]: Fetch failed at %s: %s: broker %" PRId32 + "has not yet caught up on latest metadata: " + "retrying", + rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, + rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), + rd_kafka_err2str(err), rktp->rktp_broker_id); break; case RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE: { diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 2f5ff74d6e..aed6c01dbb 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2215,8 +2215,8 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, .rkmc_full_topics_sent; if (topic_cnt == 0 && ApiVersion >= 1 && ApiVersion < 9) { - rd_kafka_buf_update_i32(rkbuf, - of_TopicArrayCnt, -1); /* Null: all topics*/ + rd_kafka_buf_update_i32(rkbuf, of_TopicArrayCnt, + -1); /* Null: all topics*/ } else { rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, topic_cnt); diff --git a/tests/0104-fetch_from_follower_mock.c b/tests/0104-fetch_from_follower_mock.c index 6749ab57b9..eb9e8e623b 100644 --- a/tests/0104-fetch_from_follower_mock.c +++ b/tests/0104-fetch_from_follower_mock.c @@ -33,6 +33,25 @@ * @name Fetch from follower tests using the mock broker. */ +static int allowed_error; + +/** + * @brief Decide what error_cb's will cause the test to fail. + */ +static int +error_is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) { + if (err == allowed_error || + /* If transport errors are allowed then it is likely + * that we'll also see ALL_BROKERS_DOWN. */ + (allowed_error == RD_KAFKA_RESP_ERR__TRANSPORT && + err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN)) { + TEST_SAY("Ignoring allowed error: %s: %s\n", + rd_kafka_err2name(err), reason); + return 0; + } + return 1; +} + /** * @brief Test offset reset when fetching from replica. @@ -248,7 +267,7 @@ static void do_test_replica_not_available(void) { const char *topic = "test"; const int msgcnt = 1000; - TEST_SAY(_C_MAG "[ Test REPLICA_NOT_AVAIALBLE ]\n"); + TEST_SAY(_C_MAG "[ Test REPLICA_NOT_AVAILABLE ]\n"); mcluster = test_mock_cluster_new(3, &bootstraps); @@ -283,7 +302,7 @@ static void do_test_replica_not_available(void) { RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE, 0); - test_consumer_assign_partition("REPLICA_NOT_AVAIALBLE", c, topic, 0, + test_consumer_assign_partition("REPLICA_NOT_AVAILABLE", c, topic, 0, RD_KAFKA_OFFSET_INVALID); test_consumer_poll_no_msgs("Wait initial metadata", c, 0, 2000); @@ -300,7 +319,207 @@ static void do_test_replica_not_available(void) { test_mock_cluster_destroy(mcluster); - TEST_SAY(_C_GRN "[ Test REPLICA_NOT_AVAIALBLE PASSED ]\n"); + TEST_SAY(_C_GRN "[ Test REPLICA_NOT_AVAILABLE PASSED ]\n"); +} + +/** + * @brief With an error \p err on a Fetch request should query for the new + * leader or preferred replica and refresh metadata. + */ +static void do_test_delegate_to_leader_on_error(rd_kafka_resp_err_t err) { + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + rd_kafka_t *c; + const char *topic = "test"; + const int msgcnt = 1000; + const char *errstr = rd_kafka_err2name(err); + + TEST_SAY(_C_MAG "[ Test %s ]\n", errstr); + + mcluster = test_mock_cluster_new(3, &bootstraps); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, 10, + "bootstrap.servers", bootstraps, + "batch.num.messages", "10", NULL); + + /* Set partition leader to broker 1. */ + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1); + + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "client.rack", "myrack"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "60000"); + test_conf_set(conf, "fetch.error.backoff.ms", "1000"); + + c = test_create_consumer("mygroup", NULL, conf, NULL); + + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1 /*Broker 1*/, 1 /*FetchRequest*/, 10, err, 0, err, 0, + err, 0, err, 0, err, 0, err, 0, err, 0, err, 0, err, 0, err, 0); + + + test_consumer_assign_partition(errstr, c, topic, 0, + RD_KAFKA_OFFSET_INVALID); + + test_consumer_poll_no_msgs("Wait initial metadata", c, 0, 2000); + + /* Switch leader to broker 2 so that metadata is updated, + * causing the consumer to start fetching from the new leader. */ + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 2); + + test_consumer_poll_timeout("Consume", c, 0, 1, 0, msgcnt, NULL, 2000); + + test_consumer_close(c); + + rd_kafka_destroy(c); + + test_mock_cluster_destroy(mcluster); + + TEST_SAY(_C_GRN "[ Test %s ]\n", errstr); +} + +/** + * @brief Test when the preferred replica is no longer a follower of the + * partition leader. We should try fetch from the leader instead. + */ +static void do_test_not_leader_or_follower(void) { + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + rd_kafka_t *c; + const char *topic = "test"; + const int msgcnt = 10; + + TEST_SAY(_C_MAG "[ Test NOT_LEADER_OR_FOLLOWER ]\n"); + + mcluster = test_mock_cluster_new(3, &bootstraps); + /* Set partition leader to broker 1. */ + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1); + rd_kafka_mock_partition_set_follower(mcluster, topic, 0, 2); + + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "client.rack", "myrack"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "60000"); + test_conf_set(conf, "fetch.error.backoff.ms", "1000"); + test_conf_set(conf, "fetch.message.max.bytes", "10"); + + c = test_create_consumer("mygroup", NULL, conf, NULL); + + test_consumer_assign_partition("NOT_LEADER_OR_FOLLOWER", c, topic, 0, + RD_KAFKA_OFFSET_INVALID); + + /* Since there are no messages, this poll only waits for metadata, and + * then sets the preferred replica after the first fetch request. */ + test_consumer_poll_no_msgs("Initial metadata and preferred replica set", + c, 0, 2000); + + /* Change the follower, so that the preferred replica is no longer the + * leader or follower. */ + rd_kafka_mock_partition_set_follower(mcluster, topic, 0, -1); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, 1000, + "bootstrap.servers", bootstraps, + "batch.num.messages", "10", NULL); + + /* On getting a NOT_LEADER_OR_FOLLOWER error, we should change to the + * leader and fetch from there without timing out. */ + test_msgver_t mv; + test_msgver_init(&mv, 0); + test_consumer_poll_timeout("from leader", c, 0, 1, 0, msgcnt, &mv, + 2000); + test_msgver_verify0( + __FUNCTION__, __LINE__, "broker_id", &mv, TEST_MSGVER_BY_BROKER_ID, + (struct test_mv_vs) { + .msg_base = 0, .exp_cnt = msgcnt, .broker_id = 1}); + test_msgver_clear(&mv); + + test_consumer_close(c); + + rd_kafka_destroy(c); + + test_mock_cluster_destroy(mcluster); + + TEST_SAY(_C_GRN "[ Test NOT_LEADER_OR_FOLLOWER PASSED ]\n"); +} + + +/** + * @brief Test when the preferred replica broker goes down. When a broker is + * going down, we should delegate all its partitions to their leaders. + */ +static void do_test_follower_down(void) { + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + rd_kafka_t *c; + const char *topic = "test"; + const int msgcnt = 10; + + TEST_SAY(_C_MAG "[ Test with follower down ]\n"); + + mcluster = test_mock_cluster_new(3, &bootstraps); + /* Set partition leader to broker 1. */ + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1); + rd_kafka_mock_partition_set_follower(mcluster, topic, 0, 2); + + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "client.rack", "myrack"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "60000"); + test_conf_set(conf, "fetch.error.backoff.ms", "1000"); + test_conf_set(conf, "fetch.message.max.bytes", "10"); + + c = test_create_consumer("mygroup", NULL, conf, NULL); + + test_consumer_assign_partition("follower down", c, topic, 0, + RD_KAFKA_OFFSET_INVALID); + + /* Since there are no messages, this poll only waits for metadata, and + * then sets the preferred replica after the first fetch request. */ + test_consumer_poll_no_msgs("Initial metadata and preferred replica set", + c, 0, 2000); + + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, 1000, + "bootstrap.servers", bootstraps, + "batch.num.messages", "10", NULL); + + /* Set follower down. When follower is set as DOWN, we also expect + * that the cluster itself knows and does not ask us to change our + * preferred replica to the broker which is down. To facilitate this, + * we just set the follower to 3 instead of 2. */ + allowed_error = RD_KAFKA_RESP_ERR__TRANSPORT; + test_curr->is_fatal_cb = error_is_fatal_cb; + rd_kafka_mock_broker_set_down(mcluster, 2); + rd_kafka_mock_partition_set_follower(mcluster, topic, 0, 3); + + /* Wee should change to the new follower when the old one goes down, + * and fetch from there without timing out. */ + test_msgver_t mv; + test_msgver_init(&mv, 0); + test_consumer_poll_timeout("from other follower", c, 0, 1, 0, msgcnt, + &mv, 2000); + test_msgver_verify0( + __FUNCTION__, __LINE__, "broker_id", &mv, TEST_MSGVER_BY_BROKER_ID, + (struct test_mv_vs) { + .msg_base = 0, .exp_cnt = msgcnt, .broker_id = 3}); + test_msgver_clear(&mv); + + test_consumer_close(c); + + rd_kafka_destroy(c); + + test_mock_cluster_destroy(mcluster); + + TEST_SAY(_C_GRN "[ Test with follower down PASSED ]\n"); } @@ -320,5 +539,12 @@ int main_0104_fetch_from_follower_mock(int argc, char **argv) { do_test_replica_not_available(); + do_test_delegate_to_leader_on_error( + RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE); + + do_test_not_leader_or_follower(); + + do_test_follower_down(); + return 0; } diff --git a/tests/test.c b/tests/test.c index 5cb71eb27e..dea4b18600 100644 --- a/tests/test.c +++ b/tests/test.c @@ -3587,6 +3587,38 @@ static int test_mv_mvec_verify_dup(test_msgver_t *mv, return fails; } +/** + * @brief Verify that all messages are from the correct broker. + */ +static int test_mv_mvec_verify_broker(test_msgver_t *mv, + int flags, + struct test_mv_p *p, + struct test_mv_mvec *mvec, + struct test_mv_vs *vs) { + int mi; + int fails = 0; + + /* Assume that the correct flag has been checked already. */ + + + rd_assert(flags & TEST_MSGVER_BY_BROKER_ID); + for (mi = 0; mi < mvec->cnt; mi++) { + struct test_mv_m *this = test_mv_mvec_get(mvec, mi); + if (this->broker_id != vs->broker_id) { + TEST_MV_WARN( + mv, + " %s [%" PRId32 + "] broker_id check: " + "msgid #%d (at mi %d): " + "broker_id %" PRId32 + " is not the expected broker_id %" PRId32 "\n", + p ? p->topic : "*", p ? p->partition : -1, + this->msgid, mi, this->broker_id, vs->broker_id); + fails++; + } + } + return fails; +} /** @@ -3896,6 +3928,10 @@ int test_msgver_verify0(const char *func, fails += test_mv_p_verify_f(mv, flags, test_mv_mvec_verify_dup, &vs); + if (flags & TEST_MSGVER_BY_BROKER_ID) + fails += test_mv_p_verify_f(mv, flags, + test_mv_mvec_verify_broker, &vs); + /* Checks across all partitions */ if ((flags & TEST_MSGVER_RANGE) && vs.exp_cnt > 0) { vs.msgid_min = vs.msg_base; @@ -4180,19 +4216,21 @@ int test_consumer_poll_once(rd_kafka_t *rk, test_msgver_t *mv, int timeout_ms) { return 1; } - /** * @param exact Require exact exp_eof_cnt (unless -1) and exp_cnt (unless -1). * If false: poll until either one is reached. + * @param timeout_ms Each call to poll has a timeout set by this argument. The + * test fails if any poll times out. */ -int test_consumer_poll_exact(const char *what, - rd_kafka_t *rk, - uint64_t testid, - int exp_eof_cnt, - int exp_msg_base, - int exp_cnt, - rd_bool_t exact, - test_msgver_t *mv) { +int test_consumer_poll_exact_timeout(const char *what, + rd_kafka_t *rk, + uint64_t testid, + int exp_eof_cnt, + int exp_msg_base, + int exp_cnt, + rd_bool_t exact, + test_msgver_t *mv, + int timeout_ms) { int eof_cnt = 0; int cnt = 0; test_timing_t t_cons; @@ -4207,7 +4245,8 @@ int test_consumer_poll_exact(const char *what, (exact && (eof_cnt < exp_eof_cnt || cnt < exp_cnt))) { rd_kafka_message_t *rkmessage; - rkmessage = rd_kafka_consumer_poll(rk, tmout_multip(10 * 1000)); + rkmessage = + rd_kafka_consumer_poll(rk, tmout_multip(timeout_ms)); if (!rkmessage) /* Shouldn't take this long to get a msg */ TEST_FAIL( "%s: consumer_poll() timeout " @@ -4269,6 +4308,23 @@ int test_consumer_poll_exact(const char *what, } +/** + * @param exact Require exact exp_eof_cnt (unless -1) and exp_cnt (unless -1). + * If false: poll until either one is reached. + */ +int test_consumer_poll_exact(const char *what, + rd_kafka_t *rk, + uint64_t testid, + int exp_eof_cnt, + int exp_msg_base, + int exp_cnt, + rd_bool_t exact, + test_msgver_t *mv) { + return test_consumer_poll_exact_timeout(what, rk, testid, exp_eof_cnt, + exp_msg_base, exp_cnt, exact, + mv, 10 * 1000); +} + int test_consumer_poll(const char *what, rd_kafka_t *rk, uint64_t testid, @@ -4281,6 +4337,19 @@ int test_consumer_poll(const char *what, rd_false /*not exact */, mv); } +int test_consumer_poll_timeout(const char *what, + rd_kafka_t *rk, + uint64_t testid, + int exp_eof_cnt, + int exp_msg_base, + int exp_cnt, + test_msgver_t *mv, + int timeout_ms) { + return test_consumer_poll_exact_timeout( + what, rk, testid, exp_eof_cnt, exp_msg_base, exp_cnt, + rd_false /*not exact */, mv, timeout_ms); +} + void test_consumer_close(rd_kafka_t *rk) { rd_kafka_resp_err_t err; test_timing_t timing; diff --git a/tests/test.h b/tests/test.h index 27f52abd57..a431f9a255 100644 --- a/tests/test.h +++ b/tests/test.h @@ -616,6 +616,15 @@ void test_consumer_poll_expect_err(rd_kafka_t *rk, int timeout_ms, rd_kafka_resp_err_t err); int test_consumer_poll_once(rd_kafka_t *rk, test_msgver_t *mv, int timeout_ms); +int test_consumer_poll_exact_timeout(const char *what, + rd_kafka_t *rk, + uint64_t testid, + int exp_eof_cnt, + int exp_msg_base, + int exp_cnt, + rd_bool_t exact, + test_msgver_t *mv, + int timeout_ms); int test_consumer_poll_exact(const char *what, rd_kafka_t *rk, uint64_t testid, @@ -631,6 +640,14 @@ int test_consumer_poll(const char *what, int exp_msg_base, int exp_cnt, test_msgver_t *mv); +int test_consumer_poll_timeout(const char *what, + rd_kafka_t *rk, + uint64_t testid, + int exp_eof_cnt, + int exp_msg_base, + int exp_cnt, + test_msgver_t *mv, + int timeout_ms); void test_consumer_wait_assignment(rd_kafka_t *rk, rd_bool_t do_poll); void test_consumer_verify_assignment0(const char *func, From 80270ad5e190d09884e52279b8736a6b2dc88f2d Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 27 Jan 2023 15:49:14 +0100 Subject: [PATCH 14/40] Fix upgrade Metadata to version 9 --- src/rdkafka_buf.h | 3 +- src/rdkafka_metadata.c | 36 +++++--------------- src/rdkafka_mock_handlers.c | 27 ++++++++++----- src/rdkafka_request.c | 67 +++++++++++++++++++++---------------- 4 files changed, 68 insertions(+), 65 deletions(-) diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h index f26905ec63..b4f606317b 100644 --- a/src/rdkafka_buf.h +++ b/src/rdkafka_buf.h @@ -815,7 +815,8 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */ } else { \ rd_kafka_buf_read_i32(rkbuf, arrcnt); \ } \ - if (*(arrcnt) < 0 || ((maxval) != -1 && *(arrcnt) > (maxval))) \ + if (*(arrcnt) < -1 || \ + ((maxval) != -1 && *(arrcnt) > (maxval))) \ rd_kafka_buf_parse_fail( \ rkbuf, "ApiArrayCnt %" PRId32 " out of range", \ *(arrcnt)); \ diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index 9967cd6778..bd7cf986db 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -422,19 +422,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, } /* Replicas */ - rd_kafka_buf_read_i32a( - rkbuf, md->topics[i].partitions[j].replica_cnt); - if (md->topics[i].partitions[j].replica_cnt > - RD_KAFKAP_BROKERS_MAX) - rd_kafka_buf_parse_fail( - rkbuf, - "TopicMetadata[%i]." - "PartitionMetadata[%i]." - "Replica_cnt " - "%i > BROKERS_MAX %i", - i, j, - md->topics[i].partitions[j].replica_cnt, - RD_KAFKAP_BROKERS_MAX); + rd_kafka_buf_read_arraycnt( + rkbuf, &md->topics[i].partitions[j].replica_cnt, + RD_KAFKAP_BROKERS_MAX); if (!(md->topics[i].partitions[j].replicas = rd_tmpabuf_alloc( @@ -460,18 +450,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, md->topics[i].partitions[j].replicas[k]); /* Isrs */ - rd_kafka_buf_read_i32a( - rkbuf, md->topics[i].partitions[j].isr_cnt); - if (md->topics[i].partitions[j].isr_cnt > - RD_KAFKAP_BROKERS_MAX) - rd_kafka_buf_parse_fail( - rkbuf, - "TopicMetadata[%i]." - "PartitionMetadata[%i]." - "Isr_cnt " - "%i > BROKERS_MAX %i", - i, j, md->topics[i].partitions[j].isr_cnt, - RD_KAFKAP_BROKERS_MAX); + rd_kafka_buf_read_arraycnt( + rkbuf, &md->topics[i].partitions[j].isr_cnt, + RD_KAFKAP_BROKERS_MAX); if (!(md->topics[i] .partitions[j] @@ -500,8 +481,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, int32_t offline_replicas_cnt; /* #OfflineReplicas */ - rd_kafka_buf_read_i32(rkbuf, - &offline_replicas_cnt); + rd_kafka_buf_read_arraycnt( + rkbuf, &offline_replicas_cnt, + RD_KAFKAP_BROKERS_MAX); rd_kafka_buf_skip(rkbuf, offline_replicas_cnt * sizeof(int32_t)); } diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index f6d29aa8ef..7b96e8e701 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -861,7 +861,7 @@ rd_kafka_mock_buf_write_Metadata_Topic(rd_kafka_buf_t *resp, rd_kafka_buf_write_bool(resp, rd_false); } /* Response: Topics.#Partitions */ - rd_kafka_buf_write_i32(resp, partition_cnt); + rd_kafka_buf_write_arraycnt(resp, partition_cnt); for (i = 0; mtopic && i < partition_cnt; i++) { const rd_kafka_mock_partition_t *mpart = &mtopic->partitions[i]; @@ -881,26 +881,30 @@ rd_kafka_mock_buf_write_Metadata_Topic(rd_kafka_buf_t *resp, } /* Response: ..Partitions.#ReplicaNodes */ - rd_kafka_buf_write_i32(resp, mpart->replica_cnt); + rd_kafka_buf_write_arraycnt(resp, mpart->replica_cnt); for (r = 0; r < mpart->replica_cnt; r++) rd_kafka_buf_write_i32(resp, mpart->replicas[r]->id); /* Response: ..Partitions.#IsrNodes */ /* Let Replicas == ISRs for now */ - rd_kafka_buf_write_i32(resp, mpart->replica_cnt); + rd_kafka_buf_write_arraycnt(resp, mpart->replica_cnt); for (r = 0; r < mpart->replica_cnt; r++) rd_kafka_buf_write_i32(resp, mpart->replicas[r]->id); if (ApiVersion >= 5) { /* Response: ...OfflineReplicas */ - rd_kafka_buf_write_i32(resp, 0); + rd_kafka_buf_write_arraycnt(resp, 0); } + + rd_kafka_buf_write_tags(resp); } if (ApiVersion >= 8) { /* Response: Topics.TopicAuthorizedOperations */ rd_kafka_buf_write_i32(resp, INT32_MIN); } + + rd_kafka_buf_write_tags(resp); } @@ -925,7 +929,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, } /* Response: #Brokers */ - rd_kafka_buf_write_i32(resp, mcluster->broker_cnt); + rd_kafka_buf_write_arraycnt(resp, mcluster->broker_cnt); TAILQ_FOREACH(mrkb, &mcluster->brokers, link) { /* Response: Brokers.Nodeid */ @@ -938,6 +942,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, /* Response: Brokers.Rack (Matt's going to love this) */ rd_kafka_buf_write_str(resp, mrkb->rack, -1); } + rd_kafka_buf_write_tags(resp); } if (rkbuf->rkbuf_reqhdr.ApiVersion >= 2) { @@ -951,7 +956,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, } /* #Topics */ - rd_kafka_buf_read_i32(rkbuf, &TopicsCnt); + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX); if (TopicsCnt > 0) requested_topics = rd_kafka_topic_partition_list_new(TopicsCnt); @@ -967,6 +972,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, rd_kafka_topic_partition_list_add(requested_topics, topic, RD_KAFKA_PARTITION_UA); + rd_kafka_buf_skip_tags(rkbuf); } if (rkbuf->rkbuf_reqhdr.ApiVersion >= 4) @@ -984,7 +990,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, if (list_all_topics) { rd_kafka_mock_topic_t *mtopic; /* Response: #Topics */ - rd_kafka_buf_write_i32(resp, mcluster->topic_cnt); + rd_kafka_buf_write_arraycnt(resp, mcluster->topic_cnt); TAILQ_FOREACH(mtopic, &mcluster->topics, link) { rd_kafka_mock_buf_write_Metadata_Topic( @@ -994,7 +1000,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, } else if (requested_topics) { /* Response: #Topics */ - rd_kafka_buf_write_i32(resp, requested_topics->cnt); + rd_kafka_buf_write_arraycnt(resp, requested_topics->cnt); for (i = 0; i < requested_topics->cnt; i++) { const rd_kafka_topic_partition_t *rktpar = @@ -1017,7 +1023,7 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, } else { /* Response: #Topics: brokers only */ - rd_kafka_buf_write_i32(resp, 0); + rd_kafka_buf_write_arraycnt(resp, 0); } if (rkbuf->rkbuf_reqhdr.ApiVersion >= 8 && @@ -1026,6 +1032,9 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, INT32_MIN); } + rd_kafka_buf_skip_tags(rkbuf); + rd_kafka_buf_write_tags(resp); + if (requested_topics) rd_kafka_topic_partition_list_destroy(requested_topics); diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index aed6c01dbb..ef384086af 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2187,7 +2187,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, int *full_incr = NULL; ApiVersion = rd_kafka_broker_ApiVersion_supported( - rkb, RD_KAFKAP_Metadata, 0, 8, &features); + rkb, RD_KAFKAP_Metadata, 0, 9, &features); rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_Metadata, 1, 4 + (50 * topic_cnt) + 1, @@ -2202,37 +2202,48 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, /* TopicArrayCnt */ of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); - if (!topics && ApiVersion >= 1) { - /* a null(0) array (in the protocol) represents no topics */ + if (!topics) { + /* v0: keep 0, brokers only not available, + * request all topics */ + /* v1-8: 0 means empty array, brokers only */ + if (ApiVersion >= 9) { + /* v9+: varint encoded empty array (1), brokers only */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, + topic_cnt); + } + rd_rkb_dbg(rkb, METADATA, "METADATA", "Request metadata for brokers only: %s", reason); full_incr = &rkb->rkb_rk->rk_metadata_cache.rkmc_full_brokers_sent; - } else { - if (topic_cnt == 0 && !rko) + } else if (topic_cnt == 0) { + /* v0: keep 0, request all topics */ + if (ApiVersion >= 1 && ApiVersion < 9) { + /* v1-8: update to -1, all topics */ + rd_kafka_buf_update_i32(rkbuf, of_TopicArrayCnt, -1); + } + /* v9+: keep 0, varint encoded null, all topics */ + + rkbuf->rkbuf_u.Metadata.all_topics = 1; + rd_rkb_dbg(rkb, METADATA, "METADATA", + "Request metadata for all topics: " + "%s", + reason); + + if (!rko) full_incr = &rkb->rkb_rk->rk_metadata_cache .rkmc_full_topics_sent; - if (topic_cnt == 0 && ApiVersion >= 1 && ApiVersion < 9) { - rd_kafka_buf_update_i32(rkbuf, of_TopicArrayCnt, - -1); /* Null: all topics*/ - } else { - rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, - topic_cnt); - } + } else { + /* request cnt topics */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, + topic_cnt); - if (topic_cnt == 0) { - rkbuf->rkbuf_u.Metadata.all_topics = 1; - rd_rkb_dbg(rkb, METADATA, "METADATA", - "Request metadata for all topics: " - "%s", - reason); - } else - rd_rkb_dbg(rkb, METADATA, "METADATA", - "Request metadata for %d topic(s): " - "%s", - topic_cnt, reason); + rd_rkb_dbg(rkb, METADATA, "METADATA", + "Request metadata for %d topic(s): " + "%s", + topic_cnt, reason); } if (full_incr) { @@ -2270,11 +2281,11 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rkbuf->rkbuf_u.Metadata.topics = rd_list_copy(topics, rd_list_string_copy, NULL); - RD_LIST_FOREACH(topic, topics, i) - rd_kafka_buf_write_str(rkbuf, topic, -1); - - /* Tags for previous topic */ - rd_kafka_buf_write_tags(rkbuf); + RD_LIST_FOREACH(topic, topics, i) { + rd_kafka_buf_write_str(rkbuf, topic, -1); + /* Tags for previous topic */ + rd_kafka_buf_write_tags(rkbuf); + } } if (ApiVersion >= 4) { From 4ac9b0d1c91dbb0a06b3f5e9afb1a2965eebd714 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 10 Feb 2023 19:02:21 +0100 Subject: [PATCH 15/40] MSVC compatibility about statement expressions --- src/rdkafka.h | 1 + src/rdkafka_admin.c | 23 ++++++++++++------ src/rdkafka_assignor.c | 9 ++++--- src/rdkafka_cgrp.c | 14 +++++++---- src/rdkafka_partition.h | 5 ++++ src/rdkafka_request.c | 45 +++++++++++++++++++++++------------ src/rdkafka_request.h | 20 ++-------------- src/rdkafka_sticky_assignor.c | 9 ++++--- src/rdkafka_txnmgr.c | 19 +++++++++------ 9 files changed, 88 insertions(+), 57 deletions(-) diff --git a/src/rdkafka.h b/src/rdkafka.h index 729158608f..c8b1ae8c67 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -92,6 +92,7 @@ typedef SSIZE_T ssize_t; #define RD_DEPRECATED __attribute__((deprecated)) #if defined(__clang__) || defined(__GNUC__) || defined(__GNUG__) +#define RD_HAS_STATEMENT_EXPRESSIONS #define RD_FORMAT(...) __attribute__((format(__VA_ARGS__))) #else #define RD_FORMAT(...) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 40a167fdd1..6aaec636d5 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -3576,10 +3576,13 @@ rd_kafka_DeleteRecordsResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_read_throttle_time(reply); - offsets = rd_kafka_buf_read_topic_partitions( - reply, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, - RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + offsets = rd_kafka_buf_read_topic_partitions(reply, 0, fields); if (!offsets) rd_kafka_buf_parse_fail(reply, "Failed to parse topic partitions"); @@ -4159,9 +4162,12 @@ rd_kafka_OffsetDeleteResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_read_throttle_time(reply); - partitions = rd_kafka_buf_read_topic_partitions( - reply, 16, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, - RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); + + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + partitions = rd_kafka_buf_read_topic_partitions(reply, 16, fields); if (!partitions) { rd_snprintf(errstr, errstr_size, "Failed to parse OffsetDeleteResponse partitions"); @@ -6445,8 +6451,11 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, /* Decreased in rd_kafka_buf_destroy */ rd_kafka_broker_keep(rkb); rd_kafka_buf_read_i16(rkbuf, &version); + const rd_kafka_topic_partition_field_t fields[] = + {RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; partitions = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, rd_false, rd_false); + rkbuf, 0, fields); rd_kafka_buf_destroy(rkbuf); if (!partitions) rd_kafka_buf_parse_fail( diff --git a/src/rdkafka_assignor.c b/src/rdkafka_assignor.c index c2aa716440..7925738455 100644 --- a/src/rdkafka_assignor.c +++ b/src/rdkafka_assignor.c @@ -144,12 +144,15 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( /* If there are no owned partitions, this is specified as an * empty array, not NULL. */ rd_kafka_buf_write_i32(rkbuf, 0); /* Topic count */ - else + else { + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; rd_kafka_buf_write_topic_partitions( rkbuf, owned_partitions, rd_false /*don't skip invalid offsets*/, - rd_false /*any offset*/, - RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); + rd_false /*any offset*/, fields); + } /* Get binary buffer and allocate a new Kafka Bytes with a copy. */ rd_slice_init_full(&rkbuf->rkbuf_reader, &rkbuf->rkbuf_buf); diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 8655033693..ffc8a38295 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -1505,8 +1505,11 @@ static void rd_kafka_cgrp_handle_SyncGroup_memberstate( rkbuf->rkbuf_rkb = rd_kafka_broker_internal(rkcg->rkcg_rk); rd_kafka_buf_read_i16(rkbuf, &Version); - if (!(assignment = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION))) + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + if (!(assignment = + rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields))) goto err_parse; rd_kafka_buf_read_bytes(rkbuf, &UserData); @@ -1799,9 +1802,12 @@ static int rd_kafka_group_MemberMetadata_consumer_read( rd_kafka_buf_read_bytes(rkbuf, &UserData); rkgm->rkgm_userdata = rd_kafkap_bytes_copy(&UserData); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; if (Version >= 1 && - !(rkgm->rkgm_owned = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION))) + !(rkgm->rkgm_owned = + rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields))) goto err; rd_kafka_buf_destroy(rkbuf); diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 8ea7b1ae36..612cc4f391 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -102,11 +102,16 @@ rd_kafka_fetch_pos_make(int64_t offset, int32_t leader_epoch) { return fetchpos; } +#ifdef RD_HAS_STATEMENT_EXPRESSIONS #define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ ({ \ rd_kafka_fetch_pos_t _fetchpos = {offset, leader_epoch}; \ _fetchpos; \ }) +#else +#define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ + rd_kafka_fetch_pos_make(offset, leader_epoch) +#endif diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index c2c2014bd9..2b8338688f 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -207,7 +207,7 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb, * * @returns a newly allocated list on success, or NULL on parse error. */ -rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions0( +rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions( rd_kafka_buf_t *rkbuf, size_t estimated_part_cnt, const rd_kafka_topic_partition_field_t *fields) { @@ -305,7 +305,7 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions0( * * @remark The \p parts list MUST be sorted. */ -int rd_kafka_buf_write_topic_partitions0( +int rd_kafka_buf_write_topic_partitions( rd_kafka_buf_t *rkbuf, const rd_kafka_topic_partition_list_t *parts, rd_bool_t skip_invalid_offsets, @@ -772,12 +772,14 @@ rd_kafka_resp_err_t rd_kafka_handle_OffsetForLeaderEpoch( if (ApiVersion >= 2) rd_kafka_buf_read_throttle_time(rkbuf); - *offsets = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, ApiVersion >= 1 ? RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH : RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, - RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET); + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + *offsets = rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields); if (!*offsets) goto err_parse; @@ -822,14 +824,16 @@ void rd_kafka_OffsetForLeaderEpochRequest( rd_kafka_topic_partition_list_sort_by_topic(parts); /* Write partition list */ - rd_kafka_buf_write_topic_partitions( - rkbuf, parts, rd_false /*include invalid offsets*/, - rd_false /*skip valid offsets */, + const rd_kafka_topic_partition_field_t fields[] = { RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, /* CurrentLeaderEpoch */ RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, /* LeaderEpoch */ - RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH); + RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + rd_kafka_buf_write_topic_partitions( + rkbuf, parts, rd_false /*include invalid offsets*/, + rd_false /*skip valid offsets */, fields); rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -1152,10 +1156,12 @@ void rd_kafka_OffsetFetchRequest(rd_kafka_broker_t *rkb, /* Write partition list, filtering out partitions with valid * offsets */ + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; PartCnt = rd_kafka_buf_write_topic_partitions( rkbuf, parts, rd_false /*include invalid offsets*/, - rd_false /*skip valid offsets */, - RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); + rd_false /*skip valid offsets */, fields); } else { rd_kafka_buf_write_arraycnt_pos(rkbuf); } @@ -1590,10 +1596,13 @@ rd_kafka_OffsetDeleteRequest(rd_kafka_broker_t *rkb, /* GroupId */ rd_kafka_buf_write_str(rkbuf, grpoffsets->group, -1); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; rd_kafka_buf_write_topic_partitions( rkbuf, grpoffsets->partitions, rd_false /*dont skip invalid offsets*/, rd_false /*any offset*/, - RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); + fields); rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); @@ -1617,10 +1626,13 @@ rd_kafka_group_MemberState_consumer_write(rd_kafka_buf_t *env_rkbuf, rkbuf = rd_kafka_buf_new(1, 100); rd_kafka_buf_write_i16(rkbuf, 0); /* Version */ rd_assert(rkgm->rkgm_assignment); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; rd_kafka_buf_write_topic_partitions( rkbuf, rkgm->rkgm_assignment, rd_false /*don't skip invalid offsets*/, rd_false /* any offset */, - RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); + fields); rd_kafka_buf_write_kbytes(rkbuf, rkgm->rkgm_userdata); /* Get pointer to binary buffer */ @@ -4006,10 +4018,13 @@ rd_kafka_DeleteRecordsRequest(rd_kafka_broker_t *rkb, rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_DeleteRecords, 1, 4 + (partitions->cnt * 100) + 4); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; rd_kafka_buf_write_topic_partitions( rkbuf, partitions, rd_false /*don't skip invalid offsets*/, - rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, - RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET); + rd_false /*any offset*/, fields); /* timeout */ op_timeout = rd_kafka_confval_get_int(&options->operation_timeout); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 2bd67dc5ba..e5d336c988 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -78,33 +78,17 @@ typedef enum { RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, } rd_kafka_topic_partition_field_t; -rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions0( +rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions( rd_kafka_buf_t *rkbuf, size_t estimated_part_cnt, const rd_kafka_topic_partition_field_t *fields); -#define rd_kafka_buf_read_topic_partitions(rkbuf, estimated_part_cnt, ...) \ - ({ \ - rd_kafka_topic_partition_field_t _f[] = { \ - __VA_ARGS__, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; \ - rd_kafka_buf_read_topic_partitions0(rkbuf, estimated_part_cnt, \ - _f); \ - }) - -int rd_kafka_buf_write_topic_partitions0( +int rd_kafka_buf_write_topic_partitions( rd_kafka_buf_t *rkbuf, const rd_kafka_topic_partition_list_t *parts, rd_bool_t skip_invalid_offsets, rd_bool_t only_invalid_offsets, const rd_kafka_topic_partition_field_t *fields); -#define rd_kafka_buf_write_topic_partitions(rkbuf, parts, skip_invalid, \ - only_invalid, ...) \ - ({ \ - rd_kafka_topic_partition_field_t _f[] = { \ - __VA_ARGS__, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; \ - rd_kafka_buf_write_topic_partitions0( \ - rkbuf, parts, skip_invalid, only_invalid, _f); \ - }) rd_kafka_resp_err_t rd_kafka_FindCoordinatorRequest(rd_kafka_broker_t *rkb, diff --git a/src/rdkafka_sticky_assignor.c b/src/rdkafka_sticky_assignor.c index 0ae8286ce6..8e76ddb14e 100644 --- a/src/rdkafka_sticky_assignor.c +++ b/src/rdkafka_sticky_assignor.c @@ -1864,9 +1864,12 @@ static rd_kafkap_bytes_t *rd_kafka_sticky_assignor_get_metadata( rkbuf = rd_kafka_buf_new(1, 100); rd_assert(state->prev_assignment != NULL); - rd_kafka_buf_write_topic_partitions( - rkbuf, state->prev_assignment, rd_false /*skip invalid offsets*/, - rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + rd_kafka_buf_write_topic_partitions(rkbuf, state->prev_assignment, + rd_false /*skip invalid offsets*/, + rd_false /*any offset*/, fields); rd_kafka_buf_write_i32(rkbuf, state->generation_id); /* Get binary buffer and allocate a new Kafka Bytes with a copy. */ diff --git a/src/rdkafka_txnmgr.c b/src/rdkafka_txnmgr.c index 864ca6afed..afbc28b71c 100644 --- a/src/rdkafka_txnmgr.c +++ b/src/rdkafka_txnmgr.c @@ -1496,9 +1496,11 @@ static void rd_kafka_txn_handle_TxnOffsetCommit(rd_kafka_t *rk, rd_kafka_buf_read_throttle_time(rkbuf); - partitions = rd_kafka_buf_read_topic_partitions( - rkbuf, 0, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, - RD_KAFKA_TOPIC_PARTITION_FIELD_ERR); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + partitions = rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields); if (!partitions) goto err_parse; @@ -1705,13 +1707,16 @@ rd_kafka_txn_send_TxnOffsetCommitRequest(rd_kafka_broker_t *rkb, } /* Write per-partition offsets list */ - cnt = rd_kafka_buf_write_topic_partitions( - rkbuf, rko->rko_u.txn.offsets, rd_true /*skip invalid offsets*/, - rd_false /*any offset*/, RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, ApiVersion >= 2 ? RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH : RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP, - RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA); + RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + cnt = rd_kafka_buf_write_topic_partitions( + rkbuf, rko->rko_u.txn.offsets, rd_true /*skip invalid offsets*/, + rd_false /*any offset*/, fields); if (!cnt) { /* No valid partition offsets, don't commit. */ rd_kafka_buf_destroy(rkbuf); From 3dfa1f393c04492cb004f5c412f51f8f21359c07 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 20 Feb 2023 13:02:27 +0100 Subject: [PATCH 16/40] Differentiate current epoch from offset epoch --- src/rdkafka.h | 31 ++++++++++++++-- src/rdkafka_fetcher.c | 8 ++--- src/rdkafka_offset.c | 23 +++++++----- src/rdkafka_partition.c | 78 +++++++++++++++++++++++++++++++---------- src/rdkafka_partition.h | 54 ++++++++++++++-------------- src/rdkafka_request.c | 25 ++++++++++--- src/rdkafka_request.h | 4 ++- 7 files changed, 156 insertions(+), 67 deletions(-) diff --git a/src/rdkafka.h b/src/rdkafka.h index c8b1ae8c67..8fd37aa65d 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -918,10 +918,10 @@ void rd_kafka_topic_partition_destroy(rd_kafka_topic_partition_t *rktpar); /** - * @brief Sets the partition's leader epoch (use -1 to clear). + * @brief Sets the offset leader epoch (use -1 to clear). * * @param rktpar Partition object. - * @param leader_epoch Partition leader epoch, use -1 to reset. + * @param leader_epoch Offset leader epoch, use -1 to reset. * * @remark See KIP-320 for more information. */ @@ -931,7 +931,7 @@ void rd_kafka_topic_partition_set_leader_epoch( int32_t leader_epoch); /** - * @returns the partition's leader epoch, if relevant and known, + * @returns the offset leader epoch, if relevant and known, * else -1. * * @param rktpar Partition object. @@ -943,6 +943,31 @@ int32_t rd_kafka_topic_partition_get_leader_epoch( const rd_kafka_topic_partition_t *rktpar); +/** + * @brief Sets the partition leader current epoch (use -1 to clear). + * + * @param rktpar Partition object. + * @param leader_epoch Partition leader current epoch, use -1 to reset. + * + * @remark See KIP-320 for more information. + */ +RD_EXPORT +void rd_kafka_topic_partition_set_current_leader_epoch( + rd_kafka_topic_partition_t *rktpar, + int32_t leader_epoch); + +/** + * @returns the partition leader current epoch, if relevant and known, + * else -1. + * + * @param rktpar Partition object. + * + * @remark See KIP-320 for more information. + */ +RD_EXPORT +int32_t rd_kafka_topic_partition_get_current_leader_epoch( + const rd_kafka_topic_partition_t *rktpar); + /** * @brief A growable list of Topic+Partitions. * diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 4c3cc8a1a9..a8886b0fe1 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -861,8 +861,7 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { if (rd_kafka_buf_ApiVersion(rkbuf) >= 9) { /* CurrentLeaderEpoch */ - rd_kafka_buf_write_i32( - rkbuf, rktp->rktp_offsets.fetch_pos.leader_epoch); + rd_kafka_buf_write_i32(rkbuf, rktp->rktp_leader_epoch); } /* FetchOffset */ @@ -878,12 +877,13 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { rd_rkb_dbg(rkb, FETCH, "FETCH", "Fetch topic %.*s [%" PRId32 "] at offset %" PRId64 - " (leader epoch %" PRId32 ", v%d)", + " (leader epoch %" PRId32 + ", current leader epoch %" PRId32 ", v%d)", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_offsets.fetch_pos.offset, rktp->rktp_offsets.fetch_pos.leader_epoch, - rktp->rktp_fetch_version); + rktp->rktp_leader_epoch, rktp->rktp_fetch_version); /* We must have a valid fetch offset when we get here */ rd_dassert(rktp->rktp_offsets.fetch_pos.offset >= 0); diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index c60732e31d..989649daf0 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -920,7 +920,7 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, rd_kafka_toppar_t *rktp = opaque; rd_kafka_topic_partition_t *rktpar; int64_t end_offset; - int32_t leader_epoch; + int32_t end_offset_leader_epoch; if (err == RD_KAFKA_RESP_ERR__DESTROY) { rd_kafka_toppar_destroy(rktp); /* Drop refcnt */ @@ -1001,11 +1001,12 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, } - rktpar = &parts->elems[0]; - end_offset = rktpar->offset; - leader_epoch = rd_kafka_topic_partition_get_leader_epoch(rktpar); + rktpar = &parts->elems[0]; + end_offset = rktpar->offset; + end_offset_leader_epoch = + rd_kafka_topic_partition_get_leader_epoch(rktpar); - if (end_offset < 0 || leader_epoch < 0) { + if (end_offset < 0 || end_offset_leader_epoch < 0) { rd_kafka_offset_reset(rktp, rd_kafka_broker_id(rkb), rktp->rktp_next_fetch_start, RD_KAFKA_RESP_ERR__LOG_TRUNCATION, @@ -1023,7 +1024,7 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, "broker end offset is %" PRId64 " (leader epoch %" PRId32 ")", rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), - end_offset, leader_epoch); + end_offset, end_offset_leader_epoch); } else { rd_kafka_error_t *error; @@ -1046,7 +1047,7 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, "attempted seek to end offset failed: %s", rd_kafka_fetch_pos2str( rktp->rktp_next_fetch_start), - end_offset, leader_epoch, + end_offset, end_offset_leader_epoch, rd_kafka_error_string(error)); rd_kafka_error_destroy(error); } @@ -1064,9 +1065,11 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, "succeeded: broker end offset %" PRId64 " (leader epoch %" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition, end_offset, leader_epoch); + rktp->rktp_partition, end_offset, + end_offset_leader_epoch); - rktp->rktp_next_fetch_start.leader_epoch = leader_epoch; + rktp->rktp_next_fetch_start.leader_epoch = + end_offset_leader_epoch; rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); } @@ -1175,6 +1178,8 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { parts, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rd_kafka_topic_partition_set_leader_epoch( rktpar, rktp->rktp_next_fetch_start.leader_epoch); + rd_kafka_topic_partition_set_current_leader_epoch( + rktpar, rktp->rktp_leader_epoch); rd_kafka_toppar_keep(rktp); /* for request opaque */ rd_rkb_dbg(rktp->rktp_leader, FETCH, "VALIDATE", diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 2847d7294d..010603a612 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -123,6 +123,7 @@ static void rd_kafka_toppar_lag_handle_Offset(rd_kafka_t *rk, */ static void rd_kafka_toppar_consumer_lag_req(rd_kafka_toppar_t *rktp) { rd_kafka_topic_partition_list_t *partitions; + rd_kafka_topic_partition_t *rktpar; if (rktp->rktp_wait_consumer_lag_resp) return; /* Previous request not finished yet */ @@ -153,9 +154,11 @@ static void rd_kafka_toppar_consumer_lag_req(rd_kafka_toppar_t *rktp) { rktp->rktp_wait_consumer_lag_resp = 1; partitions = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add( - partitions, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition) - ->offset = RD_KAFKA_OFFSET_BEGINNING; + rktpar = rd_kafka_topic_partition_list_add( + partitions, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); + rktpar->offset = RD_KAFKA_OFFSET_BEGINNING; + rd_kafka_topic_partition_set_current_leader_epoch( + rktpar, rktp->rktp_leader_epoch); /* Ask for oldest offset. The newest offset is automatically * propagated in FetchResponse.HighwaterMark. */ @@ -1281,7 +1284,7 @@ void rd_kafka_toppar_offset_fetch(rd_kafka_toppar_t *rktp, part = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add0(__FUNCTION__, __LINE__, part, rktp->rktp_rkt->rkt_topic->str, - rktp->rktp_partition, rktp); + rktp->rktp_partition, rktp, NULL); rko = rd_kafka_op_new(RD_KAFKA_OP_OFFSET_FETCH); rko->rko_rktp = rd_kafka_toppar_keep(rktp); @@ -1561,6 +1564,8 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rd_kafka_topic_partition_set_from_fetch_pos(rktpar, query_pos); + rd_kafka_topic_partition_set_current_leader_epoch( + rktpar, rktp->rktp_leader_epoch); rd_kafka_ListOffsetsRequest( rkb, offsets, @@ -2653,6 +2658,30 @@ void rd_kafka_topic_partition_set_leader_epoch( parpriv->leader_epoch = leader_epoch; } +int32_t rd_kafka_topic_partition_get_current_leader_epoch( + const rd_kafka_topic_partition_t *rktpar) { + const rd_kafka_topic_partition_private_t *parpriv; + + if (!(parpriv = rktpar->_private)) + return -1; + + return parpriv->current_leader_epoch; +} + +void rd_kafka_topic_partition_set_current_leader_epoch( + rd_kafka_topic_partition_t *rktpar, + int32_t current_leader_epoch) { + rd_kafka_topic_partition_private_t *parpriv; + + /* Avoid allocating private_t if clearing the epoch */ + if (current_leader_epoch == -1 && !rktpar->_private) + return; + + parpriv = rd_kafka_topic_partition_get_private(rktpar); + + parpriv->current_leader_epoch = current_leader_epoch; +} + /** * @brief Set offset and leader epoch from a fetchpos. */ @@ -2726,13 +2755,14 @@ void rd_kafka_topic_partition_list_destroy_free(void *ptr) { * * @returns a pointer to the added element. */ -rd_kafka_topic_partition_t * -rd_kafka_topic_partition_list_add0(const char *func, - int line, - rd_kafka_topic_partition_list_t *rktparlist, - const char *topic, - int32_t partition, - rd_kafka_toppar_t *rktp) { +rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_add0( + const char *func, + int line, + rd_kafka_topic_partition_list_t *rktparlist, + const char *topic, + int32_t partition, + rd_kafka_toppar_t *rktp, + const rd_kafka_topic_partition_private_t *parpriv) { rd_kafka_topic_partition_t *rktpar; if (rktparlist->cnt == rktparlist->size) rd_kafka_topic_partition_list_grow(rktparlist, 1); @@ -2744,9 +2774,20 @@ rd_kafka_topic_partition_list_add0(const char *func, rktpar->partition = partition; rktpar->offset = RD_KAFKA_OFFSET_INVALID; - if (rktp) - rd_kafka_topic_partition_get_private(rktpar)->rktp = - rd_kafka_toppar_keep_fl(func, line, rktp); + if (parpriv) { + rd_kafka_topic_partition_private_t *parpriv_copy = + rd_kafka_topic_partition_get_private(rktpar); + if (parpriv->rktp) { + parpriv_copy->rktp = + rd_kafka_toppar_keep_fl(func, line, parpriv->rktp); + } + parpriv_copy->leader_epoch = parpriv->leader_epoch; + parpriv_copy->current_leader_epoch = parpriv->leader_epoch; + } else if (rktp) { + rd_kafka_topic_partition_private_t *parpriv_copy = + rd_kafka_topic_partition_get_private(rktpar); + parpriv_copy->rktp = rd_kafka_toppar_keep_fl(func, line, rktp); + } return rktpar; } @@ -2757,7 +2798,7 @@ rd_kafka_topic_partition_list_add(rd_kafka_topic_partition_list_t *rktparlist, const char *topic, int32_t partition) { return rd_kafka_topic_partition_list_add0( - __FUNCTION__, __LINE__, rktparlist, topic, partition, NULL); + __FUNCTION__, __LINE__, rktparlist, topic, partition, NULL, NULL); } @@ -2798,10 +2839,9 @@ void rd_kafka_topic_partition_list_add_copy( const rd_kafka_topic_partition_t *rktpar) { rd_kafka_topic_partition_t *dst; - dst = rd_kafka_topic_partition_list_add0(__FUNCTION__, __LINE__, - rktparlist, rktpar->topic, - rktpar->partition, NULL); - + dst = rd_kafka_topic_partition_list_add0( + __FUNCTION__, __LINE__, rktparlist, rktpar->topic, + rktpar->partition, NULL, rktpar->_private); rd_kafka_topic_partition_update(dst, rktpar); } diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 612cc4f391..253065e07d 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -457,6 +457,26 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ } rktp_c; }; +/** + * @struct This is a separately allocated glue object used in + * rd_kafka_topic_partition_t._private to allow referencing both + * an rktp and/or a leader epoch. Both are optional. + * The rktp, if non-NULL, owns a refcount. + * + * This glue object is not always set in ._private, but allocated on demand + * as necessary. + */ +typedef struct rd_kafka_topic_partition_private_s { + /** Reference to a toppar. Optional, may be NULL. */ + rd_kafka_toppar_t *rktp; + /** Current Leader epoch, if known, else -1. + * this is set when the API needs to send the last epoch known + * by the client. */ + int32_t current_leader_epoch; + /** Leader epoch if known, else -1. */ + int32_t leader_epoch; +} rd_kafka_topic_partition_private_t; + /** * Check if toppar is paused (consumer). @@ -646,13 +666,14 @@ void rd_kafka_topic_partition_list_destroy_free(void *ptr); void rd_kafka_topic_partition_list_clear( rd_kafka_topic_partition_list_t *rktparlist); -rd_kafka_topic_partition_t * -rd_kafka_topic_partition_list_add0(const char *func, - int line, - rd_kafka_topic_partition_list_t *rktparlist, - const char *topic, - int32_t partition, - rd_kafka_toppar_t *rktp); +rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_add0( + const char *func, + int line, + rd_kafka_topic_partition_list_t *rktparlist, + const char *topic, + int32_t partition, + rd_kafka_toppar_t *rktp, + const rd_kafka_topic_partition_private_t *parpriv); rd_kafka_topic_partition_t *rd_kafka_topic_partition_list_upsert( rd_kafka_topic_partition_list_t *rktparlist, @@ -726,25 +747,6 @@ int rd_kafka_topic_partition_list_cmp(const void *_a, const void *_b, int (*cmp)(const void *, const void *)); - -/** - * @struct This is a separately allocated glue object used in - * rd_kafka_topic_partition_t._private to allow referencing both - * an rktp and/or a leader epoch. Both are optional. - * The rktp, if non-NULL, owns a refcount. - * - * This glue object is not always set in ._private, but allocated on demand - * as necessary. - */ -typedef struct rd_kafka_topic_partition_private_s { - /** Reference to a toppar. Optional, may be NULL. */ - rd_kafka_toppar_t *rktp; - /** Leader epoch if known, else -1. */ - int32_t leader_epoch; -} rd_kafka_topic_partition_private_t; - - - /** * @returns (and creates if necessary) the ._private glue object. */ diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 2b8338688f..5d8b82df52 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -232,9 +232,10 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions( RD_KAFKAP_STR_DUPA(&topic, &kTopic); while (PartArrayCnt-- > 0) { - int32_t Partition = -1, Epoch = -1234; - int64_t Offset = -1234; - int16_t ErrorCode = 0; + int32_t Partition = -1, Epoch = -1234, + CurrentLeaderEpoch = -1234; + int64_t Offset = -1234; + int16_t ErrorCode = 0; rd_kafka_topic_partition_t *rktpar; int fi; @@ -252,6 +253,10 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions( case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET: rd_kafka_buf_read_i64(rkbuf, &Offset); break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH: + rd_kafka_buf_read_i32( + rkbuf, &CurrentLeaderEpoch); + break; case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH: rd_kafka_buf_read_i32(rkbuf, &Epoch); break; @@ -279,6 +284,9 @@ rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions( if (Epoch != -1234) rd_kafka_topic_partition_set_leader_epoch( rktpar, Epoch); + if (CurrentLeaderEpoch != -1234) + rd_kafka_topic_partition_set_current_leader_epoch( + rktpar, CurrentLeaderEpoch); rktpar->err = ErrorCode; @@ -370,6 +378,12 @@ int rd_kafka_buf_write_topic_partitions( case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET: rd_kafka_buf_write_i64(rkbuf, rktpar->offset); break; + case RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH: + rd_kafka_buf_write_i32( + rkbuf, + rd_kafka_topic_partition_get_current_leader_epoch( + rktpar)); + break; case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH: rd_kafka_buf_write_i32( rkbuf, @@ -690,7 +704,8 @@ rd_kafka_make_ListOffsetsRequest(rd_kafka_broker_t *rkb, /* CurrentLeaderEpoch */ rd_kafka_buf_write_i32( rkbuf, - rd_kafka_topic_partition_get_leader_epoch(rktpar)); + rd_kafka_topic_partition_get_current_leader_epoch( + rktpar)); /* Time/Offset */ rd_kafka_buf_write_i64(rkbuf, rktpar->offset); @@ -827,7 +842,7 @@ void rd_kafka_OffsetForLeaderEpochRequest( const rd_kafka_topic_partition_field_t fields[] = { RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, /* CurrentLeaderEpoch */ - RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, + RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH, /* LeaderEpoch */ RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index e5d336c988..3eda6be61c 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -68,8 +68,10 @@ typedef enum { RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, /** Read/write int64_t for offset */ RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET, - /** Read/write int32_t for leader_epoch */ + /** Read/write int32_t for offset leader_epoch */ RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH, + /** Read/write int32_t for current leader_epoch */ + RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH, /** Read/write int16_t for error code */ RD_KAFKA_TOPIC_PARTITION_FIELD_ERR, /** Read/write str for metadata */ From 200c4d6105f10e56b7ebc20959be855da2cbe668 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:19:58 +0100 Subject: [PATCH 17/40] Has reliable leader epochs function --- src/rdkafka_metadata.c | 23 ++++++++++++++++++++++- src/rdkafka_metadata.h | 2 ++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index bd7cf986db..4e32e5d584 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -248,6 +248,26 @@ static void rd_kafka_parse_Metadata_update_topic( } } +/** + * @brief Only brokers with Metadata version >= 9 have reliable leader + * epochs. Before that version, leader epoch must be treated + * as missing (-1). + * + * @param rkb The broker + * @return Is this a broker version with reliable leader epochs? + * + * @locality rdkafka main thread + */ +rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) { + int features; + int16_t ApiVersion = 0; + + ApiVersion = rd_kafka_broker_ApiVersion_supported( + rkb, RD_KAFKAP_Metadata, 0, 9, &features); + + return ApiVersion >= 9; +} + /** * @brief Handle a Metadata response message. @@ -512,7 +532,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, continue; } - if (leader_epochs_size > 0 && ApiVersion < 9) { + if (leader_epochs_size > 0 && + !rd_kafka_has_reliable_leader_epochs(rkb)) { /* Prior to Kafka version 2.4 (which coincides with * Metadata version 9), the broker does not propagate * leader epoch information accurately while a diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 5782152006..53a959b8ec 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -31,6 +31,8 @@ #include "rdavl.h" +rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb); + rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, rd_kafka_buf_t *request, rd_kafka_buf_t *rkbuf, From 4658abdf7b34d7789089fa80f0b1dca8b5b9a84b Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:13:57 +0100 Subject: [PATCH 18/40] Fix cached metadata without epoch --- src/rdkafka_fetcher.c | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index a8886b0fe1..d4cf5690d0 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -861,7 +861,23 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { if (rd_kafka_buf_ApiVersion(rkbuf) >= 9) { /* CurrentLeaderEpoch */ - rd_kafka_buf_write_i32(rkbuf, rktp->rktp_leader_epoch); + if (rktp->rktp_leader_epoch < 0 && + rd_kafka_has_reliable_leader_epochs(rkb)) { + /* If current leader epoch is set to -1 and + * the broker has reliable leader epochs, + * send 0 instead, so that epoch is checked + * and optionally metadata is refreshed. + * This can happen if metadata is read initially + * without an existing topic (see + * rd_kafka_topic_metadata_update2). + * TODO: have a private metadata struct that + * stores leader epochs before topic creation. + */ + rd_kafka_buf_write_i32(rkbuf, 0); + } else { + rd_kafka_buf_write_i32(rkbuf, + rktp->rktp_leader_epoch); + } } /* FetchOffset */ From 680cf2efb2a2d85d6dc15daa2c27a4718914e559 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:21:04 +0100 Subject: [PATCH 19/40] Update fetch pos leader epoch with the message set leader epoch --- src/rdkafka_msgset_reader.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/rdkafka_msgset_reader.c b/src/rdkafka_msgset_reader.c index bcad9b0578..58779f3be6 100644 --- a/src/rdkafka_msgset_reader.c +++ b/src/rdkafka_msgset_reader.c @@ -1439,6 +1439,8 @@ rd_kafka_msgset_reader_run(rd_kafka_msgset_reader_t *msetr) { if (msetr->msetr_next_offset > rktp->rktp_offsets.fetch_pos.offset) rktp->rktp_offsets.fetch_pos.offset = msetr->msetr_next_offset; + rktp->rktp_offsets.fetch_pos.leader_epoch = msetr->msetr_leader_epoch; + rd_kafka_q_destroy_owner(&msetr->msetr_rkq); /* Skip remaining part of slice so caller can continue From 1267ed6f9205276832a58340f2f790c443e7c299 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:29:31 +0100 Subject: [PATCH 20/40] Allow calling reset with log truncation error, see rd_kafka_toppar_handle_OffsetForLeaderEpoch --- src/rdkafka_offset.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 989649daf0..f62e181095 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -800,8 +800,6 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, rd_vsnprintf(reason, sizeof(reason), fmt, ap); va_end(ap); - rd_assert(err != RD_KAFKA_RESP_ERR__LOG_TRUNCATION); - /* Enqueue op for toppar handler thread if we're on the wrong thread. */ if (!thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)) { rd_kafka_op_t *rko = From 6945ca9dc8933a32a44c53f5977a6ff110582baa Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:32:36 +0100 Subject: [PATCH 21/40] Seek with wait was causing a deadlock, because it's the main thread that should execute the seek but it's blocked waiting for the response instead. --- src/rdkafka_offset.c | 26 +++----------------------- 1 file changed, 3 insertions(+), 23 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index f62e181095..a0a76c8b4e 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1020,35 +1020,15 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, RD_KAFKA_RESP_ERR__LOG_TRUNCATION, "Partition log truncation detected at %s: " "broker end offset is %" PRId64 - " (leader epoch %" PRId32 ")", + " (offset leader epoch %" PRId32 ")", rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), end_offset, end_offset_leader_epoch); } else { - rd_kafka_error_t *error; - rd_kafka_toppar_unlock(rktp); /* Seek to the updated end offset */ - error = rd_kafka_seek_partitions(rkb->rkb_rk, parts, - RD_POLL_INFINITE); - - if (error) { - rd_kafka_offset_reset( - rktp, rd_kafka_broker_id(rkb), - rktp->rktp_next_fetch_start, - RD_KAFKA_RESP_ERR__LOG_TRUNCATION, - "Partition log truncation detected at %s: " - "broker end offset is %" PRId64 - " (leader epoch %" PRId32 - "): " - "attempted seek to end offset failed: %s", - rd_kafka_fetch_pos2str( - rktp->rktp_next_fetch_start), - end_offset, end_offset_leader_epoch, - rd_kafka_error_string(error)); - rd_kafka_error_destroy(error); - } + rd_kafka_seek_partitions(rkb->rkb_rk, parts, 0); rd_kafka_topic_partition_list_destroy(parts); rd_kafka_toppar_destroy(rktp); @@ -1061,7 +1041,7 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, "%.*s [%" PRId32 "]: offset and epoch validation " "succeeded: broker end offset %" PRId64 - " (leader epoch %" PRId32 ")", + " (offset leader epoch %" PRId32 ")", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, end_offset, end_offset_leader_epoch); From 7b5a1a4cf59198b060c4489c55df16e2409e58e5 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 28 Feb 2023 17:33:19 +0100 Subject: [PATCH 22/40] Remove write tags as it's written in rd_kafka_buf_finalize --- src/rdkafka_request.c | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 5d8b82df52..1c29c63882 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -2346,9 +2346,6 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_kafka_buf_write_bool(rkbuf, rd_false); } - /* Tags for the request */ - rd_kafka_buf_write_tags(rkbuf); - rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); /* Metadata requests are part of the important control plane From d85232381fea0735ea718ae083b6b5192cdd4538 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:36:02 +0100 Subject: [PATCH 23/40] Update next fetch position before validating --- src/rdkafka_offset.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index a0a76c8b4e..692bf22ce8 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1150,6 +1150,9 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { rd_kafka_toppar_set_fetch_state( rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT); + rd_kafka_toppar_set_next_fetch_position(rktp, + rktp->rktp_offsets.fetch_pos); + /* Construct and send OffsetForLeaderEpochRequest */ parts = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( From 2f02560ab9dc0513e48eba92c63c503cb184886e Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:34:24 +0100 Subject: [PATCH 24/40] Replace current leader epoch after offset validation, allows to retry the validation if needed --- src/rdkafka_offset.c | 6 +++--- src/rdkafka_partition.c | 11 ++++++----- src/rdkafka_partition.h | 7 +++++-- src/rdkafka_topic.c | 4 ++-- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 692bf22ce8..5a5bf7a284 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -999,8 +999,8 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, } - rktpar = &parts->elems[0]; - end_offset = rktpar->offset; + /* Validation succeeded, replace leader epoch */ + rktp->rktp_leader_epoch = rktp->rktp_next_leader_epoch; end_offset_leader_epoch = rd_kafka_topic_partition_get_leader_epoch(rktpar); @@ -1160,7 +1160,7 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { rd_kafka_topic_partition_set_leader_epoch( rktpar, rktp->rktp_next_fetch_start.leader_epoch); rd_kafka_topic_partition_set_current_leader_epoch( - rktpar, rktp->rktp_leader_epoch); + rktpar, rktp->rktp_next_leader_epoch); rd_kafka_toppar_keep(rktp); /* for request opaque */ rd_rkb_dbg(rktp->rktp_leader, FETCH, "VALIDATE", diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 010603a612..7bbd693713 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -218,11 +218,12 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, rktp = rd_calloc(1, sizeof(*rktp)); - rktp->rktp_partition = partition; - rktp->rktp_rkt = rkt; - rktp->rktp_leader_id = -1; - rktp->rktp_broker_id = -1; - rktp->rktp_leader_epoch = -1; + rktp->rktp_partition = partition; + rktp->rktp_rkt = rkt; + rktp->rktp_leader_id = -1; + rktp->rktp_broker_id = -1; + rktp->rktp_leader_epoch = -1; + rktp->rktp_next_leader_epoch = -1; rd_interval_init(&rktp->rktp_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_log_intvl); diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 253065e07d..90a8281511 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -291,8 +291,11 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ #define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \ ((fetch_state) >= RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY) - int32_t rktp_leader_epoch; /**< Last known partition leader epoch, - * or -1. */ + int32_t rktp_leader_epoch; /**< Last known partition leader epoch, + * or -1. */ + int32_t rktp_next_leader_epoch; /**< Next leader epoch, + * to replace after offset + * validation */ diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index a15d9cceae..55d561e9f1 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -683,8 +683,8 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_leader_id, rktp->rktp_leader_epoch, leader_id, leader_epoch); - rktp->rktp_leader_epoch = leader_epoch; - need_epoch_validation = rd_true; + rktp->rktp_next_leader_epoch = leader_epoch; + need_epoch_validation = rd_true; } fetching_from_follower = From ade86e62affaebf953783a61cf4296bcb1fb88ea Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:30:09 +0100 Subject: [PATCH 25/40] Force leader query parameter --- src/rdkafka_topic.c | 8 ++++---- src/rdkafka_topic.h | 6 ++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 55d561e9f1..0c15aee61c 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -1820,16 +1820,16 @@ int rd_kafka_topic_match(rd_kafka_t *rk, */ void rd_kafka_topic_leader_query0(rd_kafka_t *rk, rd_kafka_topic_t *rkt, - int do_rk_lock) { + int do_rk_lock, + rd_bool_t force) { rd_list_t topics; rd_list_init(&topics, 1, rd_free); rd_list_add(&topics, rd_strdup(rkt->rkt_topic->str)); rd_kafka_metadata_refresh_topics( - rk, NULL, &topics, rd_false /*dont force*/, - rk->rk_conf.allow_auto_create_topics, rd_false /*!cgrp_update*/, - "leader query"); + rk, NULL, &topics, force, rk->rk_conf.allow_auto_create_topics, + rd_false /*!cgrp_update*/, "leader query"); rd_list_destroy(&topics); } diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h index 29fba60037..cbed9308a7 100644 --- a/src/rdkafka_topic.h +++ b/src/rdkafka_topic.h @@ -291,9 +291,11 @@ rd_kafka_resp_err_t rd_kafka_topics_leader_query_sync(rd_kafka_t *rk, int timeout_ms); void rd_kafka_topic_leader_query0(rd_kafka_t *rk, rd_kafka_topic_t *rkt, - int do_rk_lock); + int do_rk_lock, + rd_bool_t force); #define rd_kafka_topic_leader_query(rk, rkt) \ - rd_kafka_topic_leader_query0(rk, rkt, 1 /*lock*/) + rd_kafka_topic_leader_query0(rk, rkt, 1 /*lock*/, \ + rd_false /*dont force*/) #define rd_kafka_topic_fast_leader_query(rk) \ rd_kafka_metadata_fast_leader_query(rk) From 2de43711720a3e084b6613c6c942bccd4169e95a Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:37:38 +0100 Subject: [PATCH 26/40] Check error in requested partition too --- src/rdkafka_offset.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 5a5bf7a284..63357c1627 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -936,6 +936,10 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, if (unlikely(!err && parts->cnt == 0)) err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; + if (!err) { + err = (&parts->elems[0])->err; + } + if (err) { int actions; From b3e388bc706e75ff06fba203e6fdc999a806ec04 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:40:54 +0100 Subject: [PATCH 27/40] Fix error action and allow refresh to happen in this state --- src/rdkafka_offset.c | 53 +++++++++++++++++++++++++++++++------------- 1 file changed, 37 insertions(+), 16 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 63357c1627..7e790b3933 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -974,11 +974,19 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH, RD_KAFKA_ERR_ACTION_REFRESH, RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH, + RD_KAFKA_ERR_ACTION_REFRESH, + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART, + RD_KAFKA_ERR_ACTION_REFRESH, + RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE, + RD_KAFKA_ERR_ACTION_REFRESH, + RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR, RD_KAFKA_ERR_ACTION_END); if (actions & RD_KAFKA_ERR_ACTION_REFRESH) - rd_kafka_topic_leader_query(rk, rktp->rktp_rkt); + /* Metadata refresh is ongoing, so force it */ + rd_kafka_topic_leader_query0(rk, rktp->rktp_rkt, 1, + rd_true /* force */); if (actions & RD_KAFKA_ERR_ACTION_RETRY) { /* No need for refcnt on rktp for timer opaque @@ -991,28 +999,36 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, goto done; } - /* Permanent error */ - rd_kafka_offset_reset( - rktp, rd_kafka_broker_id(rkb), - RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, - rktp->rktp_leader_epoch), - RD_KAFKA_RESP_ERR__LOG_TRUNCATION, - "Unable to validate offset and epoch: %s", - rd_kafka_err2str(err)); + if (!(actions & RD_KAFKA_ERR_ACTION_REFRESH)) { + /* Permanent error */ + rd_kafka_offset_reset( + rktp, rd_kafka_broker_id(rkb), + RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, + rktp->rktp_leader_epoch), + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Unable to validate offset and epoch: %s", + rd_kafka_err2str(err)); + } goto done; } /* Validation succeeded, replace leader epoch */ rktp->rktp_leader_epoch = rktp->rktp_next_leader_epoch; + rktpar = &parts->elems[0]; + end_offset = rktpar->offset; end_offset_leader_epoch = rd_kafka_topic_partition_get_leader_epoch(rktpar); if (end_offset < 0 || end_offset_leader_epoch < 0) { - rd_kafka_offset_reset(rktp, rd_kafka_broker_id(rkb), - rktp->rktp_next_fetch_start, - RD_KAFKA_RESP_ERR__LOG_TRUNCATION, - "Partition log truncation detected"); + rd_kafka_offset_reset( + rktp, rd_kafka_broker_id(rkb), rktp->rktp_next_fetch_start, + RD_KAFKA_RESP_ERR__LOG_TRUNCATION, + "Partition log truncation detected " + "at %s: broker end offset is %" PRId64 + " (offset leader epoch %" PRId32 ")", + rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), + end_offset, end_offset_leader_epoch); } else if (end_offset < rktp->rktp_next_fetch_start.offset) { @@ -1020,11 +1036,14 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, RD_KAFKA_OFFSET_INVALID /* auto.offset.reset=error */) { rd_kafka_offset_reset( rktp, rd_kafka_broker_id(rkb), - rktp->rktp_next_fetch_start, + RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, + rktp->rktp_leader_epoch), RD_KAFKA_RESP_ERR__LOG_TRUNCATION, "Partition log truncation detected at %s: " "broker end offset is %" PRId64 - " (offset leader epoch %" PRId32 ")", + " (offset leader epoch %" PRId32 + "). " + "Reset to INVALID.", rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), end_offset, end_offset_leader_epoch); @@ -1110,7 +1129,9 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { return; } - if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE) { + if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE && + rktp->rktp_fetch_state != + RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", "%.*s [%" PRId32 "]: skipping offset " From de379d16a1d93a01378806c91ce8c46b658a6552 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 1 Mar 2023 16:45:37 +0100 Subject: [PATCH 28/40] Change action for unknown leader epoch: retry instead of refresh, same action as fenced leader epoch. The consumer could be fetching from a follower that is not in sync --- src/rdkafka_fetcher.c | 71 ++++++++++++++++++++++++------------------- 1 file changed, 40 insertions(+), 31 deletions(-) diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index d4cf5690d0..8ee67a4205 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -183,33 +183,53 @@ static void rd_kafka_fetch_reply_handle_partition_error( switch (err) { /* Errors handled by rdkafka */ case RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE: - /* Occurs when: - * - Msg exists on broker but - * offset > HWM, or: - * - HWM is >= offset, but msg not - * yet available at that offset - * (replica is out of sync). - * - partition leader is out of sync. - * - * Handle by requesting metadata update, changing back to the - * leader, and then retrying FETCH (with backoff). - */ - rd_rkb_dbg(rkb, MSG, "FETCH", - "Topic %s [%" PRId32 - "]: %s not " - "available on broker %" PRId32 " (leader %" PRId32 - "): updating metadata and retrying", - rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), - rktp->rktp_broker_id, rktp->rktp_leader_id); - /* Continue with next case */ case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR: + case RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH: case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH: + if (err == RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE) { + /* Occurs when: + * - Msg exists on broker but + * offset > HWM, or: + * - HWM is >= offset, but msg not + * yet available at that offset + * (replica is out of sync). + * - partition leader is out of sync. + * + * Handle by requesting metadata update, changing back + * to the leader, and then retrying FETCH + * (with backoff). + */ + rd_rkb_dbg(rkb, MSG, "FETCH", + "Topic %s [%" PRId32 + "]: %s not " + "available on broker %" PRId32 + " (leader %" PRId32 + "): updating metadata and retrying", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, + rd_kafka_fetch_pos2str( + rktp->rktp_offsets.fetch_pos), + rktp->rktp_broker_id, rktp->rktp_leader_id); + } + + if (err == RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH) { + rd_rkb_dbg(rkb, MSG | RD_KAFKA_DBG_CONSUMER, "FETCH", + "Topic %s [%" PRId32 + "]: Fetch failed at %s: %s: broker %" PRId32 + "has not yet caught up on latest metadata: " + "retrying", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, + rd_kafka_fetch_pos2str( + rktp->rktp_offsets.fetch_pos), + rd_kafka_err2str(err), rktp->rktp_broker_id); + } + if (rktp->rktp_broker_id != rktp->rktp_leader_id) { rd_kafka_toppar_delegate_to_leader(rktp); } @@ -217,17 +237,6 @@ static void rd_kafka_fetch_reply_handle_partition_error( rd_kafka_toppar_leader_unavailable(rktp, "fetch", err); break; - case RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH: - rd_rkb_dbg(rkb, MSG | RD_KAFKA_DBG_CONSUMER, "FETCH", - "Topic %s [%" PRId32 - "]: Fetch failed at %s: %s: broker %" PRId32 - "has not yet caught up on latest metadata: " - "retrying", - rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, - rd_kafka_fetch_pos2str(rktp->rktp_offsets.fetch_pos), - rd_kafka_err2str(err), rktp->rktp_broker_id); - break; - case RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE: { rd_kafka_fetch_pos_t err_pos; From c70fc0d0bcaca6e0d1637b2f33a36df899e79ecf Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 2 Mar 2023 18:18:36 +0100 Subject: [PATCH 29/40] Remove next leader epoch using a fetch state check instead. Fix stale next fetch position update when fetch hasn't started yet --- src/rdkafka_offset.c | 45 +++++++++++++++++++++++------------------ src/rdkafka_partition.c | 11 +++++----- src/rdkafka_partition.h | 9 ++------- src/rdkafka_topic.c | 8 +++++--- 4 files changed, 37 insertions(+), 36 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 7e790b3933..445d85e751 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1013,10 +1013,8 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, } - /* Validation succeeded, replace leader epoch */ - rktp->rktp_leader_epoch = rktp->rktp_next_leader_epoch; - rktpar = &parts->elems[0]; - end_offset = rktpar->offset; + rktpar = &parts->elems[0]; + end_offset = rktpar->offset; end_offset_leader_epoch = rd_kafka_topic_partition_get_leader_epoch(rktpar); @@ -1142,6 +1140,28 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { return; } + + if (rktp->rktp_leader_id == -1 || !rktp->rktp_leader || + rktp->rktp_leader->rkb_source == RD_KAFKA_INTERNAL) { + rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", + "%.*s [%" PRId32 + "]: unable to perform offset " + "validation: partition leader not available", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + + rd_kafka_toppar_set_fetch_state(rktp, + RD_KAFKA_TOPPAR_FETCH_ACTIVE); + return; + } + + /* Update next fetch position, that could be stale since last + * fetch start. Only if the offset is real. */ + if (rktp->rktp_offsets.fetch_pos.offset > 0) { + rd_kafka_toppar_set_next_fetch_position( + rktp, rktp->rktp_offsets.fetch_pos); + } + /* If the fetch start position does not have an epoch set then * there is no point in doing validation. * This is the case for epoch-less seek()s or epoch-less @@ -1155,18 +1175,6 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start)); - return; - } - - if (rktp->rktp_leader_id == -1 || !rktp->rktp_leader || - rktp->rktp_leader->rkb_source == RD_KAFKA_INTERNAL) { - rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", - "%.*s [%" PRId32 - "]: unable to perform offset " - "validation: partition leader not available", - RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), - rktp->rktp_partition); - rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); return; @@ -1175,9 +1183,6 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { rd_kafka_toppar_set_fetch_state( rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT); - rd_kafka_toppar_set_next_fetch_position(rktp, - rktp->rktp_offsets.fetch_pos); - /* Construct and send OffsetForLeaderEpochRequest */ parts = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( @@ -1185,7 +1190,7 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { rd_kafka_topic_partition_set_leader_epoch( rktpar, rktp->rktp_next_fetch_start.leader_epoch); rd_kafka_topic_partition_set_current_leader_epoch( - rktpar, rktp->rktp_next_leader_epoch); + rktpar, rktp->rktp_leader_epoch); rd_kafka_toppar_keep(rktp); /* for request opaque */ rd_rkb_dbg(rktp->rktp_leader, FETCH, "VALIDATE", diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 7bbd693713..010603a612 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -218,12 +218,11 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, rktp = rd_calloc(1, sizeof(*rktp)); - rktp->rktp_partition = partition; - rktp->rktp_rkt = rkt; - rktp->rktp_leader_id = -1; - rktp->rktp_broker_id = -1; - rktp->rktp_leader_epoch = -1; - rktp->rktp_next_leader_epoch = -1; + rktp->rktp_partition = partition; + rktp->rktp_rkt = rkt; + rktp->rktp_leader_id = -1; + rktp->rktp_broker_id = -1; + rktp->rktp_leader_epoch = -1; rd_interval_init(&rktp->rktp_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_intvl); rd_interval_init(&rktp->rktp_new_lease_log_intvl); diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 90a8281511..093e2b4cb6 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -291,13 +291,8 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ #define RD_KAFKA_TOPPAR_FETCH_IS_STARTED(fetch_state) \ ((fetch_state) >= RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY) - int32_t rktp_leader_epoch; /**< Last known partition leader epoch, - * or -1. */ - int32_t rktp_next_leader_epoch; /**< Next leader epoch, - * to replace after offset - * validation */ - - + int32_t rktp_leader_epoch; /**< Last known partition leader epoch, + * or -1. */ int32_t rktp_fetch_msg_max_bytes; /* Max number of bytes to * fetch. diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 0c15aee61c..7818e196a4 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -675,7 +675,9 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, return 0; } - if (leader_epoch > rktp->rktp_leader_epoch) { + if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) + need_epoch_validation = rd_true; + else if (leader_epoch > rktp->rktp_leader_epoch) { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BROKER", "%s [%" PRId32 "]: leader %" PRId32 " epoch %" PRId32 " -> leader %" PRId32 @@ -683,8 +685,8 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_leader_id, rktp->rktp_leader_epoch, leader_id, leader_epoch); - rktp->rktp_next_leader_epoch = leader_epoch; - need_epoch_validation = rd_true; + rktp->rktp_leader_epoch = leader_epoch; + need_epoch_validation = rd_true; } fetching_from_follower = From ef3b7fdb66b58c67d2aa211d8e7d7465e09d041a Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 3 Mar 2023 12:12:50 +0100 Subject: [PATCH 30/40] Fix there's no leader epoch associated to the log start offset --- src/rdkafka_offset.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 445d85e751..e3ccea7a14 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -845,7 +845,7 @@ void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp, * target as new messages are produced. */ extra = "cached BEGINNING offset "; pos.offset = rktp->rktp_lo_offset; - pos.leader_epoch = rktp->rktp_leader_epoch; + pos.leader_epoch = -1; rd_kafka_toppar_next_offset_handle(rktp, pos); } else { From a77c2ee5e1cfa73f11bd91014638f9cc56ea1785 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 3 Mar 2023 17:22:32 +0100 Subject: [PATCH 31/40] rd_kafka_topic_partition_list_update copies epoch list, alter, delete consumer group offsets tests compare returned offsets and epochs --- src/rdkafka_partition.c | 6 +++ tests/0081-admin.c | 94 ++++++++++++++++++++++++++--------------- tests/test.c | 9 +++- 3 files changed, 73 insertions(+), 36 deletions(-) diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 010603a612..370bc21750 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -3936,6 +3936,7 @@ const char *rd_kafka_topic_partition_list_str( * - metadata * - metadata_size * - offset + * - offset leader epoch * - err * * Will only update partitions that are in both dst and src, other partitions @@ -3949,6 +3950,7 @@ void rd_kafka_topic_partition_list_update( for (i = 0; i < dst->cnt; i++) { rd_kafka_topic_partition_t *d = &dst->elems[i]; rd_kafka_topic_partition_t *s; + rd_kafka_topic_partition_private_t *s_priv, *d_priv; if (!(s = rd_kafka_topic_partition_list_find( (rd_kafka_topic_partition_list_t *)src, d->topic, @@ -3968,6 +3970,10 @@ void rd_kafka_topic_partition_list_update( memcpy((void *)d->metadata, s->metadata, s->metadata_size); } + + s_priv = rd_kafka_topic_partition_get_private(s); + d_priv = rd_kafka_topic_partition_get_private(d); + d_priv->leader_epoch = s_priv->leader_epoch; } } diff --git a/tests/0081-admin.c b/tests/0081-admin.c index ea6cab0ee7..7da2dff156 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -2961,7 +2961,7 @@ static void do_test_DeleteConsumerGroupOffsets(const char *what, TEST_CALL_ERR__( rd_kafka_committed(consumer, committed, tmout_multip(5 * 1000))); - if (test_partition_list_cmp(committed, orig_offsets)) { + if (test_partition_list_and_offsets_cmp(committed, orig_offsets)) { TEST_SAY("commit() list:\n"); test_print_partition_list(orig_offsets); TEST_SAY("committed() list:\n"); @@ -2975,18 +2975,21 @@ static void do_test_DeleteConsumerGroupOffsets(const char *what, offsets = rd_kafka_topic_partition_list_new(orig_offsets->cnt / 2); to_delete = rd_kafka_topic_partition_list_new(orig_offsets->cnt / 2); for (i = 0; i < orig_offsets->cnt; i++) { - if (i < orig_offsets->cnt / 2) - rd_kafka_topic_partition_list_add( + rd_kafka_topic_partition_t *rktpar; + if (i < orig_offsets->cnt / 2) { + rktpar = rd_kafka_topic_partition_list_add( offsets, orig_offsets->elems[i].topic, orig_offsets->elems[i].partition); - else { - rd_kafka_topic_partition_list_add( + rktpar->offset = orig_offsets->elems[i].offset; + } else { + rktpar = rd_kafka_topic_partition_list_add( to_delete, orig_offsets->elems[i].topic, orig_offsets->elems[i].partition); - rd_kafka_topic_partition_list_add( + rktpar->offset = RD_KAFKA_OFFSET_INVALID; + rktpar = rd_kafka_topic_partition_list_add( offsets, orig_offsets->elems[i].topic, - orig_offsets->elems[i].partition) - ->offset = RD_KAFKA_OFFSET_INVALID; + orig_offsets->elems[i].partition); + rktpar->offset = RD_KAFKA_OFFSET_INVALID; } } @@ -3045,7 +3048,7 @@ static void do_test_DeleteConsumerGroupOffsets(const char *what, deleted = rd_kafka_topic_partition_list_copy( rd_kafka_group_result_partitions(gres[0])); - if (test_partition_list_cmp(deleted, to_delete)) { + if (test_partition_list_and_offsets_cmp(deleted, to_delete)) { TEST_SAY("Result list:\n"); test_print_partition_list(deleted); TEST_SAY("Partitions passed to DeleteConsumerGroupOffsets:\n"); @@ -3084,9 +3087,13 @@ static void do_test_DeleteConsumerGroupOffsets(const char *what, TEST_SAY("Committed offsets after delete:\n"); test_print_partition_list(committed); - if (test_partition_list_cmp(committed, offsets)) { + rd_kafka_topic_partition_list_t *expected = offsets; + if (sub_consumer) + expected = orig_offsets; + + if (test_partition_list_and_offsets_cmp(committed, expected)) { TEST_SAY("expected list:\n"); - test_print_partition_list(offsets); + test_print_partition_list(expected); TEST_SAY("committed() list:\n"); test_print_partition_list(committed); TEST_FAIL("committed offsets don't match"); @@ -3223,11 +3230,14 @@ static void do_test_AlterConsumerGroupOffsets(const char *what, TEST_ALTER_CONSUMER_GROUP_OFFSETS_TOPIC_CNT * partitions_cnt); for (i = 0; i < TEST_ALTER_CONSUMER_GROUP_OFFSETS_TOPIC_CNT * partitions_cnt; - i++) - rd_kafka_topic_partition_list_add(orig_offsets, - topics[i / partitions_cnt], - i % partitions_cnt) - ->offset = (i + 1) * 10; + i++) { + rd_kafka_topic_partition_t *rktpar; + rktpar = rd_kafka_topic_partition_list_add( + orig_offsets, topics[i / partitions_cnt], + i % partitions_cnt); + rktpar->offset = (i + 1) * 10; + rd_kafka_topic_partition_set_leader_epoch(rktpar, 1); + } /* Commit some offsets, if topics exists */ if (create_topics) { @@ -3239,7 +3249,8 @@ static void do_test_AlterConsumerGroupOffsets(const char *what, TEST_CALL_ERR__(rd_kafka_committed(consumer, committed, tmout_multip(5 * 1000))); - if (test_partition_list_cmp(committed, orig_offsets)) { + if (test_partition_list_and_offsets_cmp(committed, + orig_offsets)) { TEST_SAY("commit() list:\n"); test_print_partition_list(orig_offsets); TEST_SAY("committed() list:\n"); @@ -3253,19 +3264,26 @@ static void do_test_AlterConsumerGroupOffsets(const char *what, offsets = rd_kafka_topic_partition_list_new(orig_offsets->cnt / 2); to_alter = rd_kafka_topic_partition_list_new(orig_offsets->cnt / 2); for (i = 0; i < orig_offsets->cnt; i++) { - if (i < orig_offsets->cnt / 2) - rd_kafka_topic_partition_list_add( + rd_kafka_topic_partition_t *rktpar; + if (i < orig_offsets->cnt / 2) { + rktpar = rd_kafka_topic_partition_list_add( offsets, orig_offsets->elems[i].topic, orig_offsets->elems[i].partition); - else { - rd_kafka_topic_partition_list_add( + rktpar->offset = orig_offsets->elems[i].offset; + rd_kafka_topic_partition_set_leader_epoch( + rktpar, rd_kafka_topic_partition_get_leader_epoch( + &orig_offsets->elems[i])); + } else { + rktpar = rd_kafka_topic_partition_list_add( to_alter, orig_offsets->elems[i].topic, - orig_offsets->elems[i].partition) - ->offset = 5; - rd_kafka_topic_partition_list_add( + orig_offsets->elems[i].partition); + rktpar->offset = 5; + rd_kafka_topic_partition_set_leader_epoch(rktpar, 2); + rktpar = rd_kafka_topic_partition_list_add( offsets, orig_offsets->elems[i].topic, - orig_offsets->elems[i].partition) - ->offset = 5; + orig_offsets->elems[i].partition); + rktpar->offset = 5; + rd_kafka_topic_partition_set_leader_epoch(rktpar, 2); } } @@ -3323,7 +3341,7 @@ static void do_test_AlterConsumerGroupOffsets(const char *what, alterd = rd_kafka_topic_partition_list_copy( rd_kafka_group_result_partitions(gres[0])); - if (test_partition_list_cmp(alterd, to_alter)) { + if (test_partition_list_and_offsets_cmp(alterd, to_alter)) { TEST_SAY("Result list:\n"); test_print_partition_list(alterd); TEST_SAY("Partitions passed to AlterConsumerGroupOffsets:\n"); @@ -3357,15 +3375,20 @@ static void do_test_AlterConsumerGroupOffsets(const char *what, TEST_CALL_ERR__(rd_kafka_committed(consumer, committed, tmout_multip(5 * 1000))); + rd_kafka_topic_partition_list_t *expected = offsets; + if (sub_consumer) { + /* Alter fails with an active consumer */ + expected = orig_offsets; + } TEST_SAY("Original committed offsets:\n"); test_print_partition_list(orig_offsets); TEST_SAY("Committed offsets after alter:\n"); test_print_partition_list(committed); - if (test_partition_list_cmp(committed, offsets)) { + if (test_partition_list_and_offsets_cmp(committed, expected)) { TEST_SAY("expected list:\n"); - test_print_partition_list(offsets); + test_print_partition_list(expected); TEST_SAY("committed() list:\n"); test_print_partition_list(committed); TEST_FAIL("committed offsets don't match"); @@ -3488,11 +3511,14 @@ static void do_test_ListConsumerGroupOffsets(const char *what, /* Commit some offsets */ orig_offsets = rd_kafka_topic_partition_list_new( TEST_LIST_CONSUMER_GROUP_OFFSETS_TOPIC_CNT * 2); - for (i = 0; i < TEST_LIST_CONSUMER_GROUP_OFFSETS_TOPIC_CNT * 2; i++) - rd_kafka_topic_partition_list_add( + for (i = 0; i < TEST_LIST_CONSUMER_GROUP_OFFSETS_TOPIC_CNT * 2; i++) { + rd_kafka_topic_partition_t *rktpar; + rktpar = rd_kafka_topic_partition_list_add( orig_offsets, topics[i / 2], - i % TEST_LIST_CONSUMER_GROUP_OFFSETS_TOPIC_CNT) - ->offset = (i + 1) * 10; + i % TEST_LIST_CONSUMER_GROUP_OFFSETS_TOPIC_CNT); + rktpar->offset = (i + 1) * 10; + rd_kafka_topic_partition_set_leader_epoch(rktpar, 2); + } TEST_CALL_ERR__(rd_kafka_commit(consumer, orig_offsets, 0 /*sync*/)); @@ -3501,7 +3527,7 @@ static void do_test_ListConsumerGroupOffsets(const char *what, TEST_CALL_ERR__( rd_kafka_committed(consumer, committed, tmout_multip(5 * 1000))); - if (test_partition_list_cmp(committed, orig_offsets)) { + if (test_partition_list_and_offsets_cmp(committed, orig_offsets)) { TEST_SAY("commit() list:\n"); test_print_partition_list(orig_offsets); TEST_SAY("committed() list:\n"); diff --git a/tests/test.c b/tests/test.c index 7074038841..c03cc61c7e 100644 --- a/tests/test.c +++ b/tests/test.c @@ -4503,10 +4503,13 @@ void test_print_partition_list( const rd_kafka_topic_partition_list_t *partitions) { int i; for (i = 0; i < partitions->cnt; i++) { - TEST_SAY(" %s [%" PRId32 "] offset %" PRId64 "%s%s\n", + TEST_SAY(" %s [%" PRId32 "] offset %" PRId64 " (epoch %" PRId32 + ") %s%s\n", partitions->elems[i].topic, partitions->elems[i].partition, partitions->elems[i].offset, + rd_kafka_topic_partition_get_leader_epoch( + &partitions->elems[i]), partitions->elems[i].err ? ": " : "", partitions->elems[i].err ? rd_kafka_err2str(partitions->elems[i].err) @@ -4566,7 +4569,9 @@ int test_partition_list_and_offsets_cmp(rd_kafka_topic_partition_list_t *al, const rd_kafka_topic_partition_t *a = &al->elems[i]; const rd_kafka_topic_partition_t *b = &bl->elems[i]; if (a->partition != b->partition || - strcmp(a->topic, b->topic) || a->offset != b->offset) + strcmp(a->topic, b->topic) || a->offset != b->offset || + rd_kafka_topic_partition_get_leader_epoch(a) != + rd_kafka_topic_partition_get_leader_epoch(b)) return -1; } From b96ee42a2e3e1b8635b0818f974d27a2a0d8555f Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 6 Mar 2023 14:56:18 +0100 Subject: [PATCH 32/40] Check offsets and epoch in test 0103, testing TxnOffsetCommit changes. --- tests/0103-transactions.c | 84 ++++++++++++++++++++++++++++++++++----- 1 file changed, 75 insertions(+), 9 deletions(-) diff --git a/tests/0103-transactions.c b/tests/0103-transactions.c index 92513e016c..eaab2f217d 100644 --- a/tests/0103-transactions.c +++ b/tests/0103-transactions.c @@ -304,7 +304,7 @@ void do_test_consumer_producer_txn(void) { const char *c1_groupid = input_topic; const char *c2_groupid = output_topic; rd_kafka_t *p1, *p2, *c1, *c2; - rd_kafka_conf_t *conf, *tmpconf, *c1_conf; + rd_kafka_conf_t *conf, *tmpconf; uint64_t testid; #define _MSGCNT (10 * 30) const int txncnt = 10; @@ -364,8 +364,7 @@ void do_test_consumer_producer_txn(void) { test_conf_set(tmpconf, "isolation.level", "read_committed"); test_conf_set(tmpconf, "auto.offset.reset", "earliest"); test_conf_set(tmpconf, "enable.auto.commit", "false"); - c1_conf = rd_kafka_conf_dup(tmpconf); - c1 = test_create_consumer(c1_groupid, NULL, tmpconf, NULL); + c1 = test_create_consumer(c1_groupid, NULL, tmpconf, NULL); test_consumer_subscribe(c1, input_topic); /* Create Producer 2 */ @@ -382,8 +381,6 @@ void do_test_consumer_producer_txn(void) { c2 = test_create_consumer(c2_groupid, NULL, tmpconf, NULL); test_consumer_subscribe(c2, output_topic); - rd_kafka_conf_destroy(conf); - /* Keep track of what messages to expect on the output topic */ test_msgver_init(&expect_mv, testid); @@ -391,9 +388,11 @@ void do_test_consumer_producer_txn(void) { int msgcnt2 = 10 * (1 + (txn % 3)); rd_kafka_message_t *msgs[_MSGCNT]; int i; - rd_bool_t do_abort = !(txn % 3); - rd_bool_t recreate_consumer = do_abort && txn == 3; - rd_kafka_topic_partition_list_t *offsets; + rd_bool_t do_abort = !(txn % 3); + rd_bool_t recreate_consumer = + (do_abort && txn == 3) || (!do_abort && txn == 2); + rd_kafka_topic_partition_list_t *offsets, + *expected_offsets = NULL; rd_kafka_resp_err_t err; rd_kafka_consumer_group_metadata_t *c1_cgmetadata; int remains = msgcnt2; @@ -452,6 +451,28 @@ void do_test_consumer_producer_txn(void) { TEST_CALL_ERROR__(rd_kafka_send_offsets_to_transaction( p2, offsets, c1_cgmetadata, -1)); + if (recreate_consumer && !do_abort) { + expected_offsets = + rd_kafka_topic_partition_list_new(offsets->cnt); + + /* Cannot use rd_kafka_topic_partition_list_copy + * as it needs to be destroyed before closing the + * consumer, because of the _private field holding + * a reference to the internal toppar */ + for (i = 0; i < offsets->cnt; i++) { + rd_kafka_topic_partition_t *rktpar = + &offsets->elems[i]; + rd_kafka_topic_partition_t *rktpar_new; + rktpar_new = rd_kafka_topic_partition_list_add( + expected_offsets, rktpar->topic, + rktpar->partition); + rktpar_new->offset = rktpar->offset; + rd_kafka_topic_partition_set_leader_epoch( + rktpar_new, + rd_kafka_topic_partition_get_leader_epoch( + rktpar)); + } + } rd_kafka_consumer_group_metadata_destroy(c1_cgmetadata); @@ -481,12 +502,57 @@ void do_test_consumer_producer_txn(void) { rd_kafka_consumer_close(c1); rd_kafka_destroy(c1); - c1 = test_create_consumer(c1_groupid, NULL, c1_conf, + tmpconf = rd_kafka_conf_dup(conf); + test_conf_set(tmpconf, "isolation.level", + "read_committed"); + test_conf_set(tmpconf, "auto.offset.reset", "earliest"); + test_conf_set(tmpconf, "enable.auto.commit", "false"); + c1 = test_create_consumer(c1_groupid, NULL, tmpconf, NULL); test_consumer_subscribe(c1, input_topic); + + + if (expected_offsets) { + rd_kafka_topic_partition_list_t + *committed_offsets = + rd_kafka_topic_partition_list_copy( + expected_offsets); + /* Set committed offsets and epochs to a + * different value before requesting them. */ + for (i = 0; i < committed_offsets->cnt; i++) { + rd_kafka_topic_partition_t *rktpar = + &committed_offsets->elems[i]; + rktpar->offset = -100; + rd_kafka_topic_partition_set_leader_epoch( + rktpar, -100); + } + + TEST_CALL_ERR__(rd_kafka_committed( + c1, committed_offsets, -1)); + + if (test_partition_list_and_offsets_cmp( + expected_offsets, committed_offsets)) { + TEST_SAY("expected list:\n"); + test_print_partition_list( + expected_offsets); + TEST_SAY("committed() list:\n"); + test_print_partition_list( + committed_offsets); + TEST_FAIL( + "committed offsets don't match"); + } + + rd_kafka_topic_partition_list_destroy( + committed_offsets); + + rd_kafka_topic_partition_list_destroy( + expected_offsets); + } } } + rd_kafka_conf_destroy(conf); + test_msgver_init(&actual_mv, testid); test_consumer_poll("Verify output topic", c2, testid, -1, 0, From 0300b2e53f4a77685d0d2b4967db5df55712935a Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 7 Mar 2023 11:15:10 +0100 Subject: [PATCH 33/40] Add leader epoch to consumer example --- examples/consumer.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/examples/consumer.c b/examples/consumer.c index 9e1eb173d6..21b27ca782 100644 --- a/examples/consumer.c +++ b/examples/consumer.c @@ -225,9 +225,10 @@ int main(int argc, char **argv) { } /* Proper message. */ - printf("Message on %s [%" PRId32 "] at offset %" PRId64 ":\n", + printf("Message on %s [%" PRId32 "] at offset %" PRId64 + " (leader epoch %" PRId32 "):\n", rd_kafka_topic_name(rkm->rkt), rkm->partition, - rkm->offset); + rkm->offset, rd_kafka_message_leader_epoch(rkm)); /* Print the message key. */ if (rkm->key && is_printable(rkm->key, rkm->key_len)) From 5e509549a805af0f2ff73995167ba46c530bbb95 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Tue, 7 Mar 2023 19:15:32 +0100 Subject: [PATCH 34/40] Validate seek requests Mock handler for OffsetForLeaderEpoch Mock test for seek with leader epoch --- src/rdkafka_int.h | 1 + src/rdkafka_mock.c | 36 ++++++++++- src/rdkafka_mock_handlers.c | 92 ++++++++++++++++++++++++++- src/rdkafka_mock_int.h | 9 ++- src/rdkafka_offset.c | 14 ++-- src/rdkafka_partition.c | 11 +++- src/rdkafka_partition.h | 18 ++++-- src/rdkafka_topic.c | 15 ++++- tests/0104-fetch_from_follower_mock.c | 67 +++++++++++++++++++ tests/test.c | 6 +- 10 files changed, 243 insertions(+), 26 deletions(-) diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h index 20e241c69c..584ff3c965 100644 --- a/src/rdkafka_int.h +++ b/src/rdkafka_int.h @@ -100,6 +100,7 @@ typedef struct rd_kafka_lwtopic_s rd_kafka_lwtopic_t; typedef struct rd_kafka_fetch_pos_s { int64_t offset; int32_t leader_epoch; + rd_bool_t validated; } rd_kafka_fetch_pos_t; diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index 251903cc97..ae7940533c 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -93,6 +93,7 @@ rd_kafka_mock_msgset_new(rd_kafka_mock_partition_t *mpart, rd_kafka_mock_msgset_t *mset; size_t totsize = sizeof(*mset) + RD_KAFKAP_BYTES_LEN(bytes); int64_t BaseOffset; + int32_t PartitionLeaderEpoch; int64_t orig_start_offset = mpart->start_offset; rd_assert(!RD_KAFKAP_BYTES_IS_NULL(bytes)); @@ -107,7 +108,8 @@ rd_kafka_mock_msgset_new(rd_kafka_mock_partition_t *mpart, mpart->follower_end_offset = mpart->end_offset; mpart->cnt++; - mset->bytes.len = bytes->len; + mset->bytes.len = bytes->len; + mset->leader_epoch = mpart->leader_epoch; mset->bytes.data = (void *)(mset + 1); @@ -118,7 +120,11 @@ rd_kafka_mock_msgset_new(rd_kafka_mock_partition_t *mpart, * actual absolute log offset. */ BaseOffset = htobe64(mset->first_offset); memcpy((void *)mset->bytes.data, &BaseOffset, sizeof(BaseOffset)); - + /* Update the base PartitionLeaderEpoch in the MessageSet with the + * actual partition leader epoch. */ + PartitionLeaderEpoch = htobe32(mset->leader_epoch); + memcpy(((char *)mset->bytes.data) + 12, &PartitionLeaderEpoch, + sizeof(PartitionLeaderEpoch)); /* Remove old msgsets until within limits */ while (mpart->cnt > 1 && @@ -387,6 +393,32 @@ rd_kafka_resp_err_t rd_kafka_mock_partition_leader_epoch_check( return RD_KAFKA_RESP_ERR_NO_ERROR; } +/** + * @brief Returns the end offset (last offset + 1) + * for the passed leader epoch in the mock partition. + * + * @param mpart The mock partition + * @param leader_epoch The leader epoch + * + * @return The end offset for the passed \p leader_epoch in \p mpart + */ +int64_t rd_kafka_mock_partition_offset_for_leader_epoch( + const rd_kafka_mock_partition_t *mpart, + int32_t leader_epoch) { + const rd_kafka_mock_msgset_t *mset = NULL; + + if (leader_epoch < 0) + return -1; + + TAILQ_FOREACH_REVERSE(mset, &mpart->msgsets, + rd_kafka_mock_msgset_tailq_s, link) { + if (mset->leader_epoch == leader_epoch) + return mset->last_offset + 1; + } + + return -1; +} + /** * @brief Automatically assign replicas for partition diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 7b96e8e701..42cf8a5ff8 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -292,7 +292,7 @@ static int rd_kafka_mock_handle_Fetch(rd_kafka_mock_connection_t *mconn, "] fetch err %s for offset %" PRId64 " mset %p, on_follower %d, " "start %" PRId64 ", end_offset %" PRId64 - " epoch %" PRId32, + ", current epoch %" PRId32, RD_KAFKAP_STR_PR(&Topic), Partition, rd_kafka_err2name(err), FetchOffset, mset, on_follower, mpart->start_offset, @@ -2018,6 +2018,94 @@ static int rd_kafka_mock_handle_EndTxn(rd_kafka_mock_connection_t *mconn, return -1; } +static int +rd_kafka_mock_handle_OffsetForLeaderEpoch(rd_kafka_mock_connection_t *mconn, + rd_kafka_buf_t *rkbuf) { + const rd_bool_t log_decode_errors = rd_true; + rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster; + rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf); + rd_kafka_resp_err_t err; + int32_t TopicsCnt, i; + + /* Response: ThrottleTimeMs */ + rd_kafka_buf_write_i32(resp, 0); + + /* #Topics */ + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX); + + /* Response: #Topics */ + rd_kafka_buf_write_arraycnt(resp, TopicsCnt); + + /* Inject error */ + err = rd_kafka_mock_next_request_error(mconn, resp); + + for (i = 0; i < TopicsCnt; i++) { + rd_kafkap_str_t Topic; + int32_t PartitionsCnt, j; + rd_kafka_mock_topic_t *mtopic; + + /* Topic */ + rd_kafka_buf_read_str(rkbuf, &Topic); + + mtopic = rd_kafka_mock_topic_find_by_kstr(mcluster, &Topic); + + /* Response: Topic */ + rd_kafka_buf_write_kstr(resp, &Topic); + + /* #Partitions */ + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionsCnt, + RD_KAFKAP_PARTITIONS_MAX); + + /* Response: #Partitions */ + rd_kafka_buf_write_arraycnt(resp, PartitionsCnt); + + for (j = 0; j < PartitionsCnt; j++) { + rd_kafka_mock_partition_t *mpart; + int32_t Partition, CurrentLeaderEpoch, LeaderEpoch; + int64_t EndOffset = -1; + + /* Partition */ + rd_kafka_buf_read_i32(rkbuf, &Partition); + /* CurrentLeaderEpoch */ + rd_kafka_buf_read_i32(rkbuf, &CurrentLeaderEpoch); + /* LeaderEpoch */ + rd_kafka_buf_read_i32(rkbuf, &LeaderEpoch); + + mpart = rd_kafka_mock_partition_find(mtopic, Partition); + if (!err && !mpart) + err = RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + + if (!err && mpart) + err = + rd_kafka_mock_partition_leader_epoch_check( + mpart, CurrentLeaderEpoch); + + if (!err && mpart) { + EndOffset = + rd_kafka_mock_partition_offset_for_leader_epoch( + mpart, LeaderEpoch); + } + + /* Response: ErrorCode */ + rd_kafka_buf_write_i16(resp, err); + /* Response: Partition */ + rd_kafka_buf_write_i32(resp, Partition); + /* Response: LeaderEpoch */ + rd_kafka_buf_write_i32(resp, LeaderEpoch); + /* Response: Partition */ + rd_kafka_buf_write_i64(resp, EndOffset); + } + } + + rd_kafka_mock_connection_send_response(mconn, resp); + + return 0; + +err_parse: + rd_kafka_buf_destroy(resp); + return -1; +} + /** * @brief Default request handlers @@ -2047,6 +2135,8 @@ const struct rd_kafka_mock_api_handler [RD_KAFKAP_TxnOffsetCommit] = {0, 3, 3, rd_kafka_mock_handle_TxnOffsetCommit}, [RD_KAFKAP_EndTxn] = {0, 1, -1, rd_kafka_mock_handle_EndTxn}, + [RD_KAFKAP_OffsetForLeaderEpoch] = + {2, 2, -1, rd_kafka_mock_handle_OffsetForLeaderEpoch}, }; diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index be02138606..ea3b6cab4d 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -217,6 +217,7 @@ typedef struct rd_kafka_mock_msgset_s { TAILQ_ENTRY(rd_kafka_mock_msgset_s) link; int64_t first_offset; /**< First offset in batch */ int64_t last_offset; /**< Last offset in batch */ + int32_t leader_epoch; /**< Msgset leader epoch */ rd_kafkap_bytes_t bytes; /* Space for bytes.data is allocated after the msgset_t */ } rd_kafka_mock_msgset_t; @@ -234,6 +235,8 @@ typedef struct rd_kafka_mock_committed_offset_s { } rd_kafka_mock_committed_offset_t; +TAILQ_HEAD(rd_kafka_mock_msgset_tailq_s, rd_kafka_mock_msgset_s); + /** * @struct Mock partition */ @@ -254,7 +257,7 @@ typedef struct rd_kafka_mock_partition_s { * in synch with end_offset */ - TAILQ_HEAD(, rd_kafka_mock_msgset_s) msgsets; + struct rd_kafka_mock_msgset_tailq_s msgsets; size_t size; /**< Total size of all .msgsets */ size_t cnt; /**< Total count of .msgsets */ size_t max_size; /**< Maximum size of all .msgsets, may be overshot. */ @@ -453,6 +456,10 @@ rd_kafka_resp_err_t rd_kafka_mock_partition_leader_epoch_check( const rd_kafka_mock_partition_t *mpart, int32_t leader_epoch); +int64_t rd_kafka_mock_partition_offset_for_leader_epoch( + const rd_kafka_mock_partition_t *mpart, + int32_t leader_epoch); + /** * @returns true if the ApiVersion is supported, else false. diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index e3ccea7a14..ee8d9836d0 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1049,7 +1049,12 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, rd_kafka_toppar_unlock(rktp); /* Seek to the updated end offset */ - rd_kafka_seek_partitions(rkb->rkb_rk, parts, 0); + rd_kafka_fetch_pos_t fetch_pos = + rd_kafka_topic_partition_get_fetch_pos(rktpar); + fetch_pos.validated = rd_true; + + rd_kafka_toppar_op_seek(rktp, fetch_pos, + RD_KAFKA_NO_REPLYQ); rd_kafka_topic_partition_list_destroy(parts); rd_kafka_toppar_destroy(rktp); @@ -1155,13 +1160,6 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { return; } - /* Update next fetch position, that could be stale since last - * fetch start. Only if the offset is real. */ - if (rktp->rktp_offsets.fetch_pos.offset > 0) { - rd_kafka_toppar_set_next_fetch_position( - rktp, rktp->rktp_offsets.fetch_pos); - } - /* If the fetch start position does not have an epoch set then * there is no point in doing validation. * This is the case for epoch-less seek()s or epoch-less diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 370bc21750..46d2fb3ed8 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -1792,9 +1792,14 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers, &rktp->rktp_offset_query_tmr, 1 /*lock*/); - - rd_kafka_toppar_next_offset_handle(rktp, pos); - + if (pos.offset <= 0 || pos.validated) { + rd_kafka_toppar_next_offset_handle(rktp, pos); + } else { + rd_kafka_toppar_set_fetch_state( + rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT); + rd_kafka_toppar_set_next_fetch_position(rktp, pos); + rd_kafka_offset_validate(rktp, "seek"); + } /* Signal back to caller thread that seek has commenced, or err */ err_reply: diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 093e2b4cb6..dffee04bbc 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -97,22 +97,28 @@ rd_kafka_fetch_pos_init(rd_kafka_fetch_pos_t *fetchpos) { const char *rd_kafka_fetch_pos2str(const rd_kafka_fetch_pos_t fetchpos); static RD_UNUSED RD_INLINE rd_kafka_fetch_pos_t -rd_kafka_fetch_pos_make(int64_t offset, int32_t leader_epoch) { - rd_kafka_fetch_pos_t fetchpos = {offset, leader_epoch}; +rd_kafka_fetch_pos_make(int64_t offset, + int32_t leader_epoch, + rd_bool_t validated) { + rd_kafka_fetch_pos_t fetchpos = {offset, leader_epoch, validated}; return fetchpos; } #ifdef RD_HAS_STATEMENT_EXPRESSIONS -#define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ +#define RD_KAFKA_FETCH_POS0(offset, leader_epoch, validated) \ ({ \ - rd_kafka_fetch_pos_t _fetchpos = {offset, leader_epoch}; \ + rd_kafka_fetch_pos_t _fetchpos = {offset, leader_epoch, \ + validated}; \ _fetchpos; \ }) #else -#define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ - rd_kafka_fetch_pos_make(offset, leader_epoch) +#define RD_KAFKA_FETCH_POS0(offset, leader_epoch, validated) \ + rd_kafka_fetch_pos_make(offset, leader_epoch, validated) #endif +#define RD_KAFKA_FETCH_POS(offset, leader_epoch) \ + RD_KAFKA_FETCH_POS0(offset, leader_epoch, rd_false) + typedef TAILQ_HEAD(rd_kafka_toppar_tqhead_s, diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 7818e196a4..d70895274e 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -671,8 +671,10 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, leader_epoch, rktp->rktp_leader_epoch); - rd_kafka_toppar_unlock(rktp); - return 0; + if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE) { + rd_kafka_toppar_unlock(rktp); + return 0; + } } if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) @@ -722,8 +724,15 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, "leader updated"); } - if (need_epoch_validation) + if (need_epoch_validation) { + /* Update next fetch position, that could be stale since last + * fetch start. Only if the offset is real. */ + if (rktp->rktp_offsets.fetch_pos.offset > 0) { + rd_kafka_toppar_set_next_fetch_position( + rktp, rktp->rktp_offsets.fetch_pos); + } rd_kafka_offset_validate(rktp, "epoch updated from metadata"); + } rd_kafka_toppar_unlock(rktp); diff --git a/tests/0104-fetch_from_follower_mock.c b/tests/0104-fetch_from_follower_mock.c index eb9e8e623b..1ecf99da3f 100644 --- a/tests/0104-fetch_from_follower_mock.c +++ b/tests/0104-fetch_from_follower_mock.c @@ -523,6 +523,71 @@ static void do_test_follower_down(void) { } +/** + * @brief When a seek is done with a leader epoch, + * the expected behavior is to validate it and + * start fetching from the end offset of that epoch if + * less than current offset. + * This is possible in case of external group offsets storage, + * associated with an unclean leader election. + */ +static void do_test_seek_to_offset_with_previous_epoch(void) { + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + rd_kafka_t *c; + const char *topic = "test"; + const int msgcnt = 10; + const size_t msgsize = 1000; + rd_kafka_topic_partition_list_t *rktpars; + rd_kafka_topic_partition_t *rktpar; + + SUB_TEST_QUICK(); + + mcluster = test_mock_cluster_new(3, &bootstraps); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, msgsize, + "bootstrap.servers", bootstraps, NULL); + + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "auto.offset.reset", "earliest"); + + c = test_create_consumer("mygroup", NULL, conf, NULL); + + test_consumer_assign_partition("zero", c, topic, 0, + RD_KAFKA_OFFSET_INVALID); + + test_consumer_poll("first", c, 0, 0, msgcnt, msgcnt, NULL); + + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 2); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, msgsize, + "bootstrap.servers", bootstraps, NULL); + + test_consumer_poll("second", c, 0, 0, msgcnt, msgcnt, NULL); + + rktpars = rd_kafka_topic_partition_list_new(1); + rktpar = rd_kafka_topic_partition_list_add(rktpars, topic, 0); + rktpar->offset = msgcnt * 2; + /* Will validate the offset at start fetching again + * from offset 'msgcnt'. */ + rd_kafka_topic_partition_set_leader_epoch(rktpar, 0); + rd_kafka_seek_partitions(c, rktpars, -1); + + test_consumer_poll("third", c, 0, 0, msgcnt, msgcnt, NULL); + + test_consumer_close(c); + rd_kafka_destroy(c); + + test_mock_cluster_destroy(mcluster); + + SUB_TEST_PASS(); +} + + int main_0104_fetch_from_follower_mock(int argc, char **argv) { if (test_needs_auth()) { @@ -546,5 +611,7 @@ int main_0104_fetch_from_follower_mock(int argc, char **argv) { do_test_follower_down(); + do_test_seek_to_offset_with_previous_epoch(); + return 0; } diff --git a/tests/test.c b/tests/test.c index c03cc61c7e..71180c8f47 100644 --- a/tests/test.c +++ b/tests/test.c @@ -4277,9 +4277,11 @@ int test_consumer_poll_exact_timeout(const char *what, TEST_SAYL(4, "%s: consumed message on %s [%" PRId32 "] " - "at offset %" PRId64 "\n", + "at offset %" PRId64 " (leader epoch %" PRId32 + ")\n", what, rd_kafka_topic_name(rkmessage->rkt), - rkmessage->partition, rkmessage->offset); + rkmessage->partition, rkmessage->offset, + rd_kafka_message_leader_epoch(rkmessage)); if (!mv || test_msgver_add_msg(rk, mv, rkmessage)) cnt++; From 4f304d735b6a77a692353d4af626a1c4b08f3b68 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 19 Mar 2023 15:41:07 +0100 Subject: [PATCH 35/40] Don't export get and set current leader epoch --- src/rdkafka.h | 26 -------------------------- src/rdkafka_partition.h | 25 +++++++++++++++++++++++++ 2 files changed, 25 insertions(+), 26 deletions(-) diff --git a/src/rdkafka.h b/src/rdkafka.h index 8fd37aa65d..2fcdff0867 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -942,32 +942,6 @@ RD_EXPORT int32_t rd_kafka_topic_partition_get_leader_epoch( const rd_kafka_topic_partition_t *rktpar); - -/** - * @brief Sets the partition leader current epoch (use -1 to clear). - * - * @param rktpar Partition object. - * @param leader_epoch Partition leader current epoch, use -1 to reset. - * - * @remark See KIP-320 for more information. - */ -RD_EXPORT -void rd_kafka_topic_partition_set_current_leader_epoch( - rd_kafka_topic_partition_t *rktpar, - int32_t leader_epoch); - -/** - * @returns the partition leader current epoch, if relevant and known, - * else -1. - * - * @param rktpar Partition object. - * - * @remark See KIP-320 for more information. - */ -RD_EXPORT -int32_t rd_kafka_topic_partition_get_current_leader_epoch( - const rd_kafka_topic_partition_t *rktpar); - /** * @brief A growable list of Topic+Partitions. * diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index dffee04bbc..a1f1f47cd9 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -768,6 +768,31 @@ rd_kafka_topic_partition_get_private(rd_kafka_topic_partition_t *rktpar) { } +/** + * @returns the partition leader current epoch, if relevant and known, + * else -1. + * + * @param rktpar Partition object. + * + * @remark See KIP-320 for more information. + */ +int32_t rd_kafka_topic_partition_get_current_leader_epoch( + const rd_kafka_topic_partition_t *rktpar); + + +/** + * @brief Sets the partition leader current epoch (use -1 to clear). + * + * @param rktpar Partition object. + * @param leader_epoch Partition leader current epoch, use -1 to reset. + * + * @remark See KIP-320 for more information. + */ +void rd_kafka_topic_partition_set_current_leader_epoch( + rd_kafka_topic_partition_t *rktpar, + int32_t leader_epoch); + + /** * @returns the partition's rktp if set (no refcnt increase), else NULL. */ From 88fc9be2fb8e44786a68be521a7f05d30b5e4a2d Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Sun, 19 Mar 2023 15:42:27 +0100 Subject: [PATCH 36/40] Fetch mock handler mimics broker behavior more closely. --- src/rdkafka_mock_handlers.c | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 42cf8a5ff8..3a004d41db 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -276,14 +276,11 @@ static int rd_kafka_mock_handle_Fetch(rd_kafka_mock_connection_t *mconn, /* Find MessageSet for FetchOffset */ if (!err && FetchOffset != mpart->end_offset) { - if (on_follower && - FetchOffset <= mpart->end_offset && - FetchOffset > mpart->follower_end_offset) - err = - RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE; - else if (!(mset = rd_kafka_mock_msgset_find( - mpart, FetchOffset, - on_follower))) + /* Kafka currently only returns + * OFFSET_NOT_AVAILABLE + * in ListOffsets calls */ + if (!(mset = rd_kafka_mock_msgset_find( + mpart, FetchOffset, on_follower))) err = RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE; rd_kafka_dbg( From 15ace0124c954de7d07677160d37f6dc187f29d9 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 22 Mar 2023 16:24:14 +0100 Subject: [PATCH 37/40] Adapt batch consumer fix to KIP-320 --- src/rdkafka_offset.c | 19 +++++++++---------- src/rdkafka_offset.h | 8 ++++---- src/rdkafka_queue.c | 21 +++++++++++++++------ 3 files changed, 28 insertions(+), 20 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 6b41d6f291..f770d87354 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1524,24 +1524,23 @@ void rd_kafka_offset_store_init(rd_kafka_toppar_t *rktp) { /** - * Update toppar app_offset and store_offset (if enabled) to the provided - * offset. - * + * Update toppar app_pos and store_offset (if enabled) to the provided + * offset and epoch. */ -void rd_kafka_update_app_offset(rd_kafka_t *rk, - rd_kafka_toppar_t *rktp, - int64_t offset, - rd_dolock_t do_lock) { +void rd_kafka_update_app_pos(rd_kafka_t *rk, + rd_kafka_toppar_t *rktp, + rd_kafka_fetch_pos_t pos, + rd_dolock_t do_lock) { if (do_lock) rd_kafka_toppar_lock(rktp); - rktp->rktp_app_offset = offset; + rktp->rktp_app_pos = pos; if (rk->rk_conf.enable_auto_offset_store) - rd_kafka_offset_store0(rktp, offset, + rd_kafka_offset_store0(rktp, pos, /* force: ignore assignment state */ rd_true, RD_DONT_LOCK); if (do_lock) rd_kafka_toppar_unlock(rktp); -} \ No newline at end of file +} diff --git a/src/rdkafka_offset.h b/src/rdkafka_offset.h index b8eba02161..7b01c84877 100644 --- a/src/rdkafka_offset.h +++ b/src/rdkafka_offset.h @@ -127,9 +127,9 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) void rd_kafka_offset_query_tmr_cb(rd_kafka_timers_t *rkts, void *arg); -void rd_kafka_update_app_offset(rd_kafka_t *rk, - rd_kafka_toppar_t *rktp, - int64_t offset, - rd_dolock_t do_lock); +void rd_kafka_update_app_pos(rd_kafka_t *rk, + rd_kafka_toppar_t *rktp, + rd_kafka_fetch_pos_t pos, + rd_dolock_t do_lock); #endif /* _RDKAFKA_OFFSET_H_ */ diff --git a/src/rdkafka_queue.c b/src/rdkafka_queue.c index cc46ab4b1d..57fce36b8d 100644 --- a/src/rdkafka_queue.c +++ b/src/rdkafka_queue.c @@ -684,9 +684,13 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq, rko = (rd_kafka_op_t *)rkmessages[i]->_private; rd_kafka_toppar_t *rktp = rko->rko_rktp; int64_t offset = rkmessages[i]->offset + 1; - if (unlikely(rktp->rktp_app_offset < offset)) - rd_kafka_update_app_offset(rk, rktp, offset, - RD_DO_LOCK); + if (unlikely(rktp->rktp_app_pos.offset < offset)) + rd_kafka_update_app_pos( + rk, rktp, + RD_KAFKA_FETCH_POS( + offset, + rd_kafka_message_leader_epoch(rkmessages[i])), + RD_DO_LOCK); } /* Discard non-desired and already handled ops */ @@ -704,9 +708,14 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq, next = TAILQ_NEXT(next, rko_link); rd_kafka_toppar_t *rktp = rko->rko_rktp; int64_t offset = rko->rko_u.fetch.rkm.rkm_rkmessage.offset + 1; - if (rktp->rktp_app_offset < offset) - rd_kafka_update_app_offset(rk, rktp, offset, - RD_DO_LOCK); + if (rktp->rktp_app_pos.offset < offset) + rd_kafka_update_app_pos( + rk, rktp, + RD_KAFKA_FETCH_POS( + offset, + rd_kafka_message_leader_epoch( + &rko->rko_u.fetch.rkm.rkm_rkmessage)), + RD_DO_LOCK); rd_kafka_op_destroy(rko); } From 19195a7887921899df17369db7f7c51b35497375 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 20 Mar 2023 20:12:11 +0100 Subject: [PATCH 38/40] Set app position before validating instead of fetch position --- src/rdkafka_topic.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index d70895274e..89bfa092df 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -726,10 +726,10 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt, if (need_epoch_validation) { /* Update next fetch position, that could be stale since last - * fetch start. Only if the offset is real. */ - if (rktp->rktp_offsets.fetch_pos.offset > 0) { + * fetch start. Only if the app pos is real. */ + if (rktp->rktp_app_pos.offset > 0) { rd_kafka_toppar_set_next_fetch_position( - rktp, rktp->rktp_offsets.fetch_pos); + rktp, rktp->rktp_app_pos); } rd_kafka_offset_validate(rktp, "epoch updated from metadata"); } From 8a5c46a5883f9acdca0c3147dd271e6dcdaafc82 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 22 Mar 2023 17:54:11 +0100 Subject: [PATCH 39/40] librdkafka version v2.1.0 --- configure.self | 2 +- src-cpp/rdkafkacpp.h | 2 +- src/rdkafka.h | 2 +- vcpkg.json | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/configure.self b/configure.self index 81bd7d0baf..bb0a975c94 100644 --- a/configure.self +++ b/configure.self @@ -34,7 +34,7 @@ mkl_toggle_option "Development" ENABLE_VALGRIND "--enable-valgrind" "Enable in-c mkl_toggle_option "Development" ENABLE_REFCNT_DEBUG "--enable-refcnt-debug" "Enable refcnt debugging" "n" -mkl_toggle_option "Feature" ENABLE_LZ4_EXT "--enable-lz4-ext" "Enable external LZ4 library support (builtin version 1.9.2)" "y" +mkl_toggle_option "Feature" ENABLE_LZ4_EXT "--enable-lz4-ext" "Enable external LZ4 library support (builtin version 1.9.3)" "y" mkl_toggle_option "Feature" ENABLE_LZ4_EXT "--enable-lz4" "Deprecated: alias for --enable-lz4-ext" "y" mkl_toggle_option "Feature" ENABLE_REGEX_EXT "--enable-regex-ext" "Enable external (libc) regex (else use builtin)" "y" diff --git a/src-cpp/rdkafkacpp.h b/src-cpp/rdkafkacpp.h index 343b9735d9..1df1043c07 100644 --- a/src-cpp/rdkafkacpp.h +++ b/src-cpp/rdkafkacpp.h @@ -111,7 +111,7 @@ namespace RdKafka { * @remark This value should only be used during compile time, * for runtime checks of version use RdKafka::version() */ -#define RD_KAFKA_VERSION 0x020002ff +#define RD_KAFKA_VERSION 0x020100ff /** * @brief Returns the librdkafka version as integer. diff --git a/src/rdkafka.h b/src/rdkafka.h index 2fcdff0867..e3474e50ff 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -166,7 +166,7 @@ typedef SSIZE_T ssize_t; * @remark This value should only be used during compile time, * for runtime checks of version use rd_kafka_version() */ -#define RD_KAFKA_VERSION 0x020002ff +#define RD_KAFKA_VERSION 0x020100ff /** * @brief Returns the librdkafka version as integer. diff --git a/vcpkg.json b/vcpkg.json index 5cd630e267..79ca6ab060 100644 --- a/vcpkg.json +++ b/vcpkg.json @@ -1,6 +1,6 @@ { "name": "librdkafka", - "version": "2.0.2", + "version": "2.1.0", "dependencies": [ { "name": "zstd", From 8a324bf3105c3808c1fd578e23ee910f25e6e3db Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Wed, 22 Mar 2023 19:17:45 +0100 Subject: [PATCH 40/40] Better message in the case no offset is found in OffsetForLeaderEpoch --- src/rdkafka_offset.c | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index f770d87354..ffa6a9d524 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -1022,9 +1022,11 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, rd_kafka_offset_reset( rktp, rd_kafka_broker_id(rkb), rktp->rktp_next_fetch_start, RD_KAFKA_RESP_ERR__LOG_TRUNCATION, - "Partition log truncation detected " - "at %s: broker end offset is %" PRId64 - " (offset leader epoch %" PRId32 ")", + "No epoch found less or equal to " + "%s: broker end offset is %" PRId64 + " (offset leader epoch %" PRId32 + ")." + " Reset using configured policy.", rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), end_offset, end_offset_leader_epoch);