diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 275bb76aaf..8ef7183aa1 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -126,7 +126,7 @@ blocks: - make -j -C tests build - make -C tests run_local_quick - DESTDIR="$PWD/dest" make install - - (cd tests && python3 -m trivup.clusters.KafkaCluster --version 3.1.0 --cmd 'make quick') + - (cd tests && python3 -m trivup.clusters.KafkaCluster --version 3.4.0 --cmd 'make quick') - name: 'Linux x64: release artifact docker builds' diff --git a/CHANGELOG.md b/CHANGELOG.md index 857526c6eb..d01f80dbe9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,82 @@ +# librdkafka v2.2.0 + +librdkafka v2.2.0 is a feature release: + + * Fix a segmentation fault when subscribing to non-existent topics and + using the consume batch functions (#4273). + * Store offset commit metadata in `rd_kafka_offsets_store` (@mathispesch, #4084). + * Fix a bug that happens when skipping tags, causing buffer underflow in + MetadataResponse (#4278). + * [KIP-881](https://cwiki.apache.org/confluence/display/KAFKA/KIP-881%3A+Rack-aware+Partition+Assignment+for+Kafka+Consumers): + Add support for rack-aware partition assignment for consumers + (#4184, #4291, #4252). + * Fix several bugs with sticky assignor in case of partition ownership + changing between members of the consumer group (#4252). + + +## Fixes + +### General fixes + + * Fix a bug that happens when skipping tags, causing buffer underflow in + MetadataResponse. This is triggered since RPC version 9 (v2.1.0), + when using Confluent Platform, only when racks are set, + observers are activated and there is more than one partition. + Fixed by skipping the correct amount of bytes when tags are received. + + +### Consumer fixes + + * In case of multiple owners of a partition with different generations, the + sticky assignor would pick the earliest (lowest generation) member as the + current owner, which would lead to stickiness violations. Fixed by + choosing the latest (highest generation) member. + * In case where the same partition is owned by two members with the same + generation, it indicates an issue. The sticky assignor had some code to + handle this, but it was non-functional, and did not have parity with the + Java assignor. Fixed by invalidating any such partition from the current + assignment completely. + + +# librdkafka v2.1.1 + +librdkafka v2.1.1 is a maintenance release: + + * Avoid duplicate messages when a fetch response is received + in the middle of an offset validation request (#4261). + * Fix segmentation fault when subscribing to a non-existent topic and + calling `rd_kafka_message_leader_epoch()` on the polled `rkmessage` (#4245). + * Fix a segmentation fault when fetching from follower and the partition lease + expires while waiting for the result of a list offsets operation (#4254). + * Fix documentation for the admin request timeout, incorrectly stating -1 for infinite + timeout. That timeout can't be infinite. + * Fix CMake pkg-config cURL require and use + pkg-config `Requires.private` field (@FantasqueX, @stertingen, #4180). + * Fixes certain cases where polling would not keep the consumer + in the group or make it rejoin it (#4256). + * Fix to the C++ set_leader_epoch method of TopicPartitionImpl, + that wasn't storing the passed value (@pavel-pimenov, #4267). + +## Fixes + +### Consumer fixes + + * Duplicate messages can be emitted when a fetch response is received + in the middle of an offset validation request. Solved by avoiding + a restart from last application offset when offset validation succeeds. + * When fetching from follower, if the partition lease expires after 5 minutes, + and a list offsets operation was requested to retrieve the earliest + or latest offset, it resulted in segmentation fault. This was fixed by + allowing threads different from the main one to call + the `rd_kafka_toppar_set_fetch_state` function, given they hold + the lock on the `rktp`. + * In v2.1.0, a bug was fixed which caused polling any queue to reset the + `max.poll.interval.ms`. Only certain functions were made to reset the timer, + but it is possible for the user to obtain the queue with messages from + the broker, skipping these functions. This was fixed by encoding information + in a queue itself, that, whether polling, resets the timer. + + # librdkafka v2.1.0 librdkafka v2.1.0 is a feature release: @@ -64,11 +143,18 @@ librdkafka v2.1.0 is a feature release: any of the **seek**, **pause**, **resume** or **rebalancing** operation, `on_consume` interceptors might be called incorrectly (maybe multiple times) for not consumed messages. +### Consume API + + * Duplicate messages can be emitted when a fetch response is received + in the middle of an offset validation request. + * Segmentation fault when subscribing to a non-existent topic and + calling `rd_kafka_message_leader_epoch()` on the polled `rkmessage`. + # librdkafka v2.0.2 -librdkafka v2.0.2 is a bugfix release: +librdkafka v2.0.2 is a maintenance release: * Fix OpenSSL version in Win32 nuget package (#4152). @@ -76,7 +162,7 @@ librdkafka v2.0.2 is a bugfix release: # librdkafka v2.0.1 -librdkafka v2.0.1 is a bugfix release: +librdkafka v2.0.1 is a maintenance release: * Fixed nuget package for Linux ARM64 release (#4150). diff --git a/INTRODUCTION.md b/INTRODUCTION.md index 66f796bcab..000e454156 100644 --- a/INTRODUCTION.md +++ b/INTRODUCTION.md @@ -1957,6 +1957,7 @@ The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/conf | KIP-654 - Aborted txns with non-flushed msgs should not be fatal | 2.7.0 | Supported | | KIP-735 - Increase default consumer session timeout | 3.0.0 | Supported | | KIP-768 - SASL/OAUTHBEARER OIDC support | 3.0 | Supported | +| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers | 3.5.0 (WIP) | Supported | diff --git a/packaging/cmake/rdkafka.pc.in b/packaging/cmake/rdkafka.pc.in index 0eb17e8560..9632cf5134 100644 --- a/packaging/cmake/rdkafka.pc.in +++ b/packaging/cmake/rdkafka.pc.in @@ -6,7 +6,7 @@ libdir=${prefix}/lib Name: @PKG_CONFIG_NAME@ Description: @PKG_CONFIG_DESCRIPTION@ Version: @PKG_CONFIG_VERSION@ -Requires: @PKG_CONFIG_REQUIRES@ +Requires.private: @PKG_CONFIG_REQUIRES_PRIVATE@ Cflags: @PKG_CONFIG_CFLAGS@ Libs: @PKG_CONFIG_LIBS@ Libs.private: @PKG_CONFIG_LIBS_PRIVATE@ diff --git a/src-cpp/CMakeLists.txt b/src-cpp/CMakeLists.txt index b0a6d51e47..2b496d9f9e 100644 --- a/src-cpp/CMakeLists.txt +++ b/src-cpp/CMakeLists.txt @@ -41,7 +41,7 @@ set(PKG_CONFIG_VERSION "${PROJECT_VERSION}") if(NOT RDKAFKA_BUILD_STATIC) set(PKG_CONFIG_NAME "librdkafka++") set(PKG_CONFIG_DESCRIPTION "The Apache Kafka C/C++ library") - set(PKG_CONFIG_REQUIRES "rdkafka") + set(PKG_CONFIG_REQUIRES_PRIVATE "rdkafka") set(PKG_CONFIG_CFLAGS "-I\${includedir}") set(PKG_CONFIG_LIBS "-L\${libdir} -lrdkafka++") set(PKG_CONFIG_LIBS_PRIVATE "-lrdkafka") @@ -57,7 +57,7 @@ if(NOT RDKAFKA_BUILD_STATIC) else() set(PKG_CONFIG_NAME "librdkafka++-static") set(PKG_CONFIG_DESCRIPTION "The Apache Kafka C/C++ library (static)") - set(PKG_CONFIG_REQUIRES "") + set(PKG_CONFIG_REQUIRES_PRIVATE "") set(PKG_CONFIG_CFLAGS "-I\${includedir} -DLIBRDKAFKA_STATICLIB") set(PKG_CONFIG_LIBS "-L\${libdir} \${libdir}/librdkafka++.a") if(WIN32) diff --git a/src-cpp/HandleImpl.cpp b/src-cpp/HandleImpl.cpp index 7aa2f2939b..356af369bf 100644 --- a/src-cpp/HandleImpl.cpp +++ b/src-cpp/HandleImpl.cpp @@ -391,6 +391,12 @@ 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->metadata_.size()) { + void *metadata_p = mem_malloc(tpi->metadata_.size()); + memcpy(metadata_p, tpi->metadata_.data(), tpi->metadata_.size()); + rktpar->metadata = metadata_p; + rktpar->metadata_size = tpi->metadata_.size(); + } if (tpi->leader_epoch_ != -1) rd_kafka_topic_partition_set_leader_epoch(rktpar, tpi->leader_epoch_); } @@ -417,6 +423,10 @@ void update_partitions_from_c_parts( pp->offset_ = p->offset; pp->err_ = static_cast(p->err); pp->leader_epoch_ = rd_kafka_topic_partition_get_leader_epoch(p); + if (p->metadata_size) { + unsigned char *metadata = (unsigned char *)p->metadata; + pp->metadata_.assign(metadata, metadata + p->metadata_size); + } } } } diff --git a/src-cpp/rdkafkacpp.h b/src-cpp/rdkafkacpp.h index 1df1043c07..33befcaf38 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 0x020100ff +#define RD_KAFKA_VERSION 0x020101ff /** * @brief Returns the librdkafka version as integer. @@ -1986,6 +1986,12 @@ class RD_EXPORT TopicPartition { /** @brief Set partition leader epoch. */ virtual void set_leader_epoch(int32_t leader_epoch) = 0; + + /** @brief Get partition metadata. */ + virtual std::vector get_metadata() = 0; + + /** @brief Set partition metadata. */ + virtual void set_metadata(std::vector &metadata) = 0; }; diff --git a/src-cpp/rdkafkacpp_int.h b/src-cpp/rdkafkacpp_int.h index bc024ebe90..d6db4f33b7 100644 --- a/src-cpp/rdkafkacpp_int.h +++ b/src-cpp/rdkafkacpp_int.h @@ -1260,7 +1260,10 @@ class TopicPartitionImpl : public TopicPartition { offset_ = c_part->offset; err_ = static_cast(c_part->err); leader_epoch_ = rd_kafka_topic_partition_get_leader_epoch(c_part); - // FIXME: metadata + if (c_part->metadata_size > 0) { + unsigned char *metadata = (unsigned char *)c_part->metadata; + metadata_.assign(metadata, metadata + c_part->metadata_size); + } } static void destroy(std::vector &partitions); @@ -1289,7 +1292,15 @@ class TopicPartitionImpl : public TopicPartition { } void set_leader_epoch(int32_t leader_epoch) { - leader_epoch_ = leader_epoch_; + leader_epoch_ = leader_epoch; + } + + std::vector get_metadata() { + return metadata_; + } + + void set_metadata(std::vector &metadata) { + metadata_ = metadata; } std::ostream &operator<<(std::ostream &ostrm) const { @@ -1301,6 +1312,7 @@ class TopicPartitionImpl : public TopicPartition { int64_t offset_; ErrorCode err_; int32_t leader_epoch_; + std::vector metadata_; }; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 37b43c4996..33481ba1ac 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -199,7 +199,7 @@ target_include_directories(rdkafka PUBLIC "$") if(WITH_CURL) find_package(CURL REQUIRED) - target_include_directories(rdkafka PUBLIC ${CURL_INCLUDE_DIRS}) + target_include_directories(rdkafka PRIVATE ${CURL_INCLUDE_DIRS}) target_link_libraries(rdkafka PUBLIC ${CURL_LIBRARIES}) endif() @@ -272,7 +272,7 @@ endif() # Generate pkg-config file set(PKG_CONFIG_VERSION "${PROJECT_VERSION}") -set(PKG_CONFIG_REQUIRES "") +set(PKG_CONFIG_REQUIRES_PRIVATE "") if (WIN32) set(PKG_CONFIG_LIBS_PRIVATE "-lws2_32 -lsecur32 -lcrypt32") else() @@ -296,27 +296,27 @@ if(NOT RDKAFKA_BUILD_STATIC) set(PKG_CONFIG_DESCRIPTION "The Apache Kafka C/C++ library") if(WITH_CURL) - string(APPEND PKG_CONFIG_REQUIRES "curl ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "libcurl ") endif() if(WITH_ZLIB) - string(APPEND PKG_CONFIG_REQUIRES "zlib ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "zlib ") endif() if(WITH_SSL) - string(APPEND PKG_CONFIG_REQUIRES "libssl ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "libcrypto libssl ") endif() if(WITH_SASL_CYRUS) - string(APPEND PKG_CONFIG_REQUIRES "libsasl2 ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "libsasl2 ") endif() if(WITH_ZSTD) - string(APPEND PKG_CONFIG_REQUIRES "libzstd ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "libzstd ") endif() if(WITH_LZ4_EXT) - string(APPEND PKG_CONFIG_REQUIRES "liblz4 ") + string(APPEND PKG_CONFIG_REQUIRES_PRIVATE "liblz4 ") endif() set(PKG_CONFIG_CFLAGS "-I\${includedir}") diff --git a/src/rdkafka.c b/src/rdkafka.c index d0125cd1d2..ebccb5afb5 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -990,7 +990,7 @@ void rd_kafka_destroy_final(rd_kafka_t *rk) { mtx_destroy(&rk->rk_init_lock); if (rk->rk_full_metadata) - rd_kafka_metadata_destroy(rk->rk_full_metadata); + rd_kafka_metadata_destroy(&rk->rk_full_metadata->metadata); rd_kafkap_str_destroy(rk->rk_client_id); rd_kafkap_str_destroy(rk->rk_group_id); rd_kafkap_str_destroy(rk->rk_eos.transactional_id); @@ -4003,20 +4003,37 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk, int rd_kafka_poll(rd_kafka_t *rk, int timeout_ms) { int r; + const rd_bool_t can_q_contain_fetched_msgs = + rd_kafka_q_can_contain_fetched_msgs(rk->rk_rep, RD_DO_LOCK); + + if (timeout_ms && can_q_contain_fetched_msgs) + rd_kafka_app_poll_blocking(rk); r = rd_kafka_q_serve(rk->rk_rep, timeout_ms, 0, RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb, NULL); + if (can_q_contain_fetched_msgs) + rd_kafka_app_polled(rk); + return r; } rd_kafka_event_t *rd_kafka_queue_poll(rd_kafka_queue_t *rkqu, int timeout_ms) { rd_kafka_op_t *rko; + const rd_bool_t can_q_contain_fetched_msgs = + rd_kafka_q_can_contain_fetched_msgs(rkqu->rkqu_q, RD_DO_LOCK); + + + if (timeout_ms && can_q_contain_fetched_msgs) + rd_kafka_app_poll_blocking(rkqu->rkqu_rk); rko = rd_kafka_q_pop_serve(rkqu->rkqu_q, rd_timeout_us(timeout_ms), 0, RD_KAFKA_Q_CB_EVENT, rd_kafka_poll_cb, NULL); + if (can_q_contain_fetched_msgs) + rd_kafka_app_polled(rkqu->rkqu_rk); + if (!rko) return NULL; @@ -4025,10 +4042,18 @@ rd_kafka_event_t *rd_kafka_queue_poll(rd_kafka_queue_t *rkqu, int timeout_ms) { int rd_kafka_queue_poll_callback(rd_kafka_queue_t *rkqu, int timeout_ms) { int r; + const rd_bool_t can_q_contain_fetched_msgs = + rd_kafka_q_can_contain_fetched_msgs(rkqu->rkqu_q, RD_DO_LOCK); + + if (timeout_ms && can_q_contain_fetched_msgs) + rd_kafka_app_poll_blocking(rkqu->rkqu_rk); r = rd_kafka_q_serve(rkqu->rkqu_q, timeout_ms, 0, RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb, NULL); + if (can_q_contain_fetched_msgs) + rd_kafka_app_polled(rkqu->rkqu_rk); + return r; } diff --git a/src/rdkafka.h b/src/rdkafka.h index 24b2e84ce2..d26924c415 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 0x020100ff +#define RD_KAFKA_VERSION 0x020101ff /** * @brief Returns the librdkafka version as integer. @@ -3431,6 +3431,12 @@ rd_kafka_error_t *rd_kafka_sasl_set_credentials(rd_kafka_t *rk, * * @remark rd_kafka_queue_destroy() MUST be called on this queue * prior to calling rd_kafka_consumer_close(). + * @remark Polling the returned queue counts as a consumer poll, and will reset + * the timer for max.poll.interval.ms. If this queue is forwarded to a + * "destq", polling destq also counts as a consumer poll (this works + * for any number of forwards). However, even if this queue is + * unforwarded or forwarded elsewhere, polling destq will continue + * to count as a consumer poll. */ RD_EXPORT rd_kafka_queue_t *rd_kafka_queue_get_consumer(rd_kafka_t *rk); @@ -6808,8 +6814,7 @@ RD_EXPORT void rd_kafka_AdminOptions_destroy(rd_kafka_AdminOptions_t *options); * request transmission, operation time on broker, and response. * * @param options Admin options. - * @param timeout_ms Timeout in milliseconds, use -1 for indefinite timeout. - * Defaults to `socket.timeout.ms`. + * @param timeout_ms Timeout in milliseconds. Defaults to `socket.timeout.ms`. * @param errstr A human readable error string (nul-terminated) is written to * this location that must be of at least \p errstr_size bytes. * The \p errstr is only written in case of error. diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 4de2c3e7fd..305f5c016b 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -6473,7 +6473,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, char *errstr, size_t errstr_size) { const int log_decode_errors = LOG_ERR; - int nodeid; + int32_t nodeid; uint16_t port; int16_t api_version; int32_t cnt; @@ -7104,7 +7104,7 @@ rd_kafka_admin_DescribeTopicsRequest(rd_kafka_broker_t *rkb, err = rd_kafka_MetadataRequest(rkb, topics, "describe topics", rd_false, rd_false, include_topic_authorized_operations, - rd_false, NULL, resp_cb, 0, opaque); + rd_false, rd_false, NULL, resp_cb, 0, opaque); if (err) { rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); @@ -7123,28 +7123,24 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_t *reply, char *errstr, size_t errstr_size) { + rd_kafka_metadata_internal_t *mdi = NULL; struct rd_kafka_metadata *md = NULL; rd_kafka_resp_err_t err; rd_list_t topics = rko_req->rko_u.admin_request.args; rd_kafka_broker_t *rkb = reply->rkbuf_rkb; rd_kafka_topic_authorized_operations_pair_t *topic_authorized_operations = NULL; - int32_t cluster_authorized_operations; - char *cluster_id = NULL; - int controller_id; int i, cnt; rd_kafka_op_t *rko_result = NULL; // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || // thrd_is_current(rk->rk_thread)); // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); - err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, - &topic_authorized_operations, - &cluster_authorized_operations, &topics, - &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &mdi, &topics); if (err) goto err; rko_result = rd_kafka_admin_result_new(rko_req); + md = &mdi->metadata; rd_list_init(&rko_result->rko_u.admin_result.results, md->topic_cnt, rd_kafka_TopicDescription_free); cnt = md->topic_cnt; @@ -7154,14 +7150,13 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req, /* topics in md should be in the same order as in * topic_authorized_operations*/ rd_assert(strcmp(md->topics[i].topic, - topic_authorized_operations[i].topic_name) == + mdi->topics[i].topic_name) == 0); if (md->topics[i].err == RD_KAFKA_RESP_ERR_NO_ERROR) { rd_list_t *authorized_operations; authorized_operations = rd_kafka_AuthorizedOperations_parse( - topic_authorized_operations[i] - .authorized_operations); + mdi->topics[i].topic_authorized_operations); topicdesc = rd_kafka_TopicDescription_new( md->topics[i].topic, md->topics[i].partitions, md->topics[i].partition_cnt, authorized_operations, @@ -7472,10 +7467,13 @@ rd_kafka_admin_DescribeClusterRequest(rd_kafka_broker_t *rkb, /* resp_cb = rd_kafka_admin_handle_response; */ // err = Call metadata request with NULL topics - err = rd_kafka_MetadataRequest(rkb, NULL, "describe cluster", rd_false, + err = rd_kafka_MetadataRequest(rkb, NULL, "describe cluster", + rd_false /*no auto create*/, include_cluster_authorized_operations, - rd_false, rd_false, NULL, resp_cb, 1, - opaque); + rd_false /*!include topic authorized operations */, + rd_false /*cgrp update*/, + rd_false /* force_rack */, + NULL, resp_cb, 1, opaque); if (err) { rd_snprintf(errstr, errstr_size, "%s", rd_kafka_err2str(err)); @@ -7493,7 +7491,7 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_buf_t *reply, char *errstr, size_t errstr_size) { - struct rd_kafka_metadata *md = NULL; + rd_kafka_metadata_internal_t *mdi = NULL; rd_kafka_resp_err_t err; rd_kafka_ClusterDescription_t *clusterdesc = NULL; rd_list_t topics = rko_req->rko_u.admin_request.args; @@ -7509,10 +7507,10 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, // thrd_is_current(rk->rk_thread)); // rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY); - err = rd_kafka_parse_Metadata(rkb, NULL, reply, &md, - &topic_authorized_operations, - &cluster_authorized_operations, &topics, - &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata(rkb, NULL, reply, &mdi, &topics); + cluster_id = mdi->cluster_id; + controller_id = mdi->controller_id; + cluster_authorized_operations = mdi->cluster_authorized_operations; if (err) goto err; rko_result = rd_kafka_admin_result_new(rko_req); @@ -7524,7 +7522,7 @@ rd_kafka_DescribeClusterResponse_parse(rd_kafka_op_t *rko_req, rd_kafka_AuthorizedOperations_parse(cluster_authorized_operations); clusterdesc = rd_kafka_ClusterDescription_new( - cluster_id, controller_id, authorized_operations, md); + cluster_id, controller_id, authorized_operations, &mdi->metadata); if (authorized_operations) rd_list_destroy(authorized_operations); rd_free(cluster_id); diff --git a/src/rdkafka_assignment.c b/src/rdkafka_assignment.c index dc4bdae947..2afc648193 100644 --- a/src/rdkafka_assignment.c +++ b/src/rdkafka_assignment.c @@ -341,6 +341,8 @@ static int rd_kafka_assignment_serve_removals(rd_kafka_t *rk) { * so it will be committed below. */ rd_kafka_topic_partition_set_from_fetch_pos( rktpar, rktp->rktp_stored_pos); + rd_kafka_topic_partition_set_metadata_from_rktp_stored(rktpar, + rktp); valid_offsets += !RD_KAFKA_OFFSET_IS_LOGICAL(rktpar->offset); /* Reset the stored offset to invalid so that @@ -348,8 +350,8 @@ static int rd_kafka_assignment_serve_removals(rd_kafka_t *rk) { * will not commit a stored offset from a previous * assignment (issue #2782). */ rd_kafka_offset_store0( - rktp, RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), - rd_true, RD_DONT_LOCK); + rktp, RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), NULL, + 0, rd_true, RD_DONT_LOCK); /* Partition is no longer desired */ rd_kafka_toppar_desired_del(rktp); @@ -745,8 +747,8 @@ 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_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), - rd_true /* force */, RD_DONT_LOCK); + rktp, RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1), NULL, + 0, rd_true /* force */, RD_DONT_LOCK); rd_kafka_toppar_unlock(rktp); } diff --git a/src/rdkafka_assignor.c b/src/rdkafka_assignor.c index 7925738455..fe5615ad7c 100644 --- a/src/rdkafka_assignor.c +++ b/src/rdkafka_assignor.c @@ -59,6 +59,9 @@ void rd_kafka_group_member_clear(rd_kafka_group_member_t *rkgm) { if (rkgm->rkgm_member_metadata) rd_kafkap_bytes_destroy(rkgm->rkgm_member_metadata); + if (rkgm->rkgm_rack_id) + rd_kafkap_str_destroy(rkgm->rkgm_rack_id); + memset(rkgm, 0, sizeof(*rkgm)); } @@ -106,7 +109,9 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( const rd_list_t *topics, const void *userdata, size_t userdata_size, - const rd_kafka_topic_partition_list_t *owned_partitions) { + const rd_kafka_topic_partition_list_t *owned_partitions, + int generation, + const rd_kafkap_str_t *rack_id) { rd_kafka_buf_t *rkbuf; rd_kafkap_bytes_t *kbytes; @@ -124,12 +129,14 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( * OwnedPartitions => [Topic Partitions] // added in v1 * Topic => string * Partitions => [int32] + * GenerationId => int32 // added in v2 + * RackId => string // added in v3 */ rkbuf = rd_kafka_buf_new(1, 100 + (topic_cnt * 100) + userdata_size); /* Version */ - rd_kafka_buf_write_i16(rkbuf, 1); + rd_kafka_buf_write_i16(rkbuf, 3); rd_kafka_buf_write_i32(rkbuf, topic_cnt); RD_LIST_FOREACH(tinfo, topics, i) rd_kafka_buf_write_str(rkbuf, tinfo->topic, -1); @@ -154,6 +161,12 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( rd_false /*any offset*/, fields); } + /* Following data is ignored by consumer version < 2 */ + rd_kafka_buf_write_i32(rkbuf, generation); + + /* Following data is ignored by consumer version < 3 */ + rd_kafka_buf_write_kstr(rkbuf, rack_id); + /* Get binary buffer and allocate a new Kafka Bytes with a copy. */ rd_slice_init_full(&rkbuf->rkbuf_reader, &rkbuf->rkbuf_buf); len = rd_slice_remains(&rkbuf->rkbuf_reader); @@ -170,9 +183,13 @@ rd_kafkap_bytes_t *rd_kafka_assignor_get_metadata_with_empty_userdata( const rd_kafka_assignor_t *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions) { - return rd_kafka_consumer_protocol_member_metadata_new(topics, NULL, 0, - owned_partitions); + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id) { + /* Generation was earlier populated inside userData, and older versions + * of clients still expect that. So, in case the userData is empty, we + * set the explicit generation field to the default value, -1 */ + return rd_kafka_consumer_protocol_member_metadata_new( + topics, NULL, 0, owned_partitions, -1 /* generation */, rack_id); } @@ -244,6 +261,8 @@ rd_kafka_member_subscriptions_map(rd_kafka_cgrp_t *rkcg, int member_cnt) { int ti; rd_kafka_assignor_topic_t *eligible_topic = NULL; + rd_kafka_metadata_internal_t *mdi = + rd_kafka_metadata_get_internal(metadata); rd_list_init(eligible_topics, RD_MIN(metadata->topic_cnt, 10), (void *)rd_kafka_assignor_topic_destroy); @@ -285,7 +304,8 @@ rd_kafka_member_subscriptions_map(rd_kafka_cgrp_t *rkcg, continue; } - eligible_topic->metadata = &metadata->topics[ti]; + eligible_topic->metadata = &metadata->topics[ti]; + eligible_topic->metadata_internal = &mdi->topics[ti]; rd_list_add(eligible_topics, eligible_topic); eligible_topic = NULL; } @@ -485,7 +505,8 @@ rd_kafka_resp_err_t rd_kafka_assignor_add( const struct rd_kafka_assignor_s *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions), + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id), void (*on_assignment_cb)(const struct rd_kafka_assignor_s *rkas, void **assignor_state, const rd_kafka_topic_partition_list_t *assignment, @@ -636,6 +657,676 @@ void rd_kafka_assignors_term(rd_kafka_t *rk) { rd_list_destroy(&rk->rk_conf.partition_assignors); } +/** + * @brief Computes whether rack-aware assignment needs to be used, or not. + */ +rd_bool_t +rd_kafka_use_rack_aware_assignment(rd_kafka_assignor_topic_t **topics, + size_t topic_cnt, + const rd_kafka_metadata_internal_t *mdi) { + /* Computing needs_rack_aware_assignment requires the evaluation of + three criteria: + + 1. At least one of the member has a non-null rack. + 2. At least one common rack exists between members and partitions. + 3. There is a partition which doesn't have replicas on all possible + racks, or in other words, all partitions don't have replicas on all + racks. Note that 'all racks' here means racks across all replicas of + all partitions, not including consumer racks. Also note that 'all + racks' are computed per-topic for range assignor, and across topics + for sticky assignor. + */ + + int i; + size_t t; + rd_kafka_group_member_t *member; + rd_list_t *all_consumer_racks = NULL; /* Contained Type: char* */ + rd_list_t *all_partition_racks = NULL; /* Contained Type: char* */ + char *rack_id = NULL; + rd_bool_t needs_rack_aware_assignment = rd_true; /* assume true */ + + /* Criteria 1 */ + /* We don't copy racks, so the free function is NULL. */ + all_consumer_racks = rd_list_new(0, NULL); + + for (t = 0; t < topic_cnt; t++) { + RD_LIST_FOREACH(member, &topics[t]->members, i) { + if (member->rkgm_rack_id && + RD_KAFKAP_STR_LEN(member->rkgm_rack_id)) { + /* Repetitions are fine, we will dedup it later. + */ + rd_list_add( + all_consumer_racks, + /* The const qualifier has to be discarded + because of how rd_list_t and + rd_kafkap_str_t are, but we never modify + items in all_consumer_racks. */ + (char *)member->rkgm_rack_id->str); + } + } + } + if (rd_list_cnt(all_consumer_racks) == 0) { + needs_rack_aware_assignment = rd_false; + goto done; + } + + + /* Critera 2 */ + /* We don't copy racks, so the free function is NULL. */ + all_partition_racks = rd_list_new(0, NULL); + + for (t = 0; t < topic_cnt; t++) { + const int partition_cnt = topics[t]->metadata->partition_cnt; + for (i = 0; i < partition_cnt; i++) { + size_t j; + for (j = 0; j < topics[t] + ->metadata_internal->partitions[i] + .racks_cnt; + j++) { + char *rack = + topics[t] + ->metadata_internal->partitions[i] + .racks[j]; + rd_list_add(all_partition_racks, rack); + } + } + } + + /* If there are no partition racks, Criteria 2 cannot possibly be met. + */ + if (rd_list_cnt(all_partition_racks) == 0) { + needs_rack_aware_assignment = rd_false; + goto done; + } + + /* Sort and dedup the racks. */ + rd_list_deduplicate(&all_consumer_racks, rd_strcmp2); + rd_list_deduplicate(&all_partition_racks, rd_strcmp2); + + + /* Iterate through each list in order, and see if there's anything in + * common */ + RD_LIST_FOREACH(rack_id, all_consumer_racks, i) { + /* Break if there's even a single match. */ + if (rd_list_find(all_partition_racks, rack_id, rd_strcmp2)) { + break; + } + } + if (i == rd_list_cnt(all_consumer_racks)) { + needs_rack_aware_assignment = rd_false; + goto done; + } + + /* Criteria 3 */ + for (t = 0; t < topic_cnt; t++) { + const int partition_cnt = topics[t]->metadata->partition_cnt; + for (i = 0; i < partition_cnt; i++) { + /* Since partition_racks[i] is a subset of + * all_partition_racks, and both of them are deduped, + * the same size indicates that they're equal. */ + if ((size_t)(rd_list_cnt(all_partition_racks)) != + topics[t] + ->metadata_internal->partitions[i] + .racks_cnt) { + break; + } + } + if (i < partition_cnt) { + /* Break outer loop if inner loop was broken. */ + break; + } + } + + /* Implies that all partitions have replicas on all racks. */ + if (t == topic_cnt) + needs_rack_aware_assignment = rd_false; + +done: + RD_IF_FREE(all_consumer_racks, rd_list_destroy); + RD_IF_FREE(all_partition_racks, rd_list_destroy); + + return needs_rack_aware_assignment; +} + + +/* Helper to populate the racks for brokers in the metadata for unit tests. + * Passing num_broker_racks = 0 will return NULL racks. */ +void ut_populate_internal_broker_metadata(rd_kafka_metadata_internal_t *mdi, + int num_broker_racks, + rd_kafkap_str_t *all_racks[], + size_t all_racks_cnt) { + int i; + + rd_assert(num_broker_racks < (int)all_racks_cnt); + + for (i = 0; i < mdi->metadata.broker_cnt; i++) { + mdi->brokers[i].id = i; + /* Cast from const to non-const. We don't intend to modify it, + * but unfortunately neither implementation of rd_kafkap_str_t + * or rd_kafka_metadata_broker_internal_t can be changed. So, + * this cast is used - in unit tests only. */ + mdi->brokers[i].rack_id = + (char *)(num_broker_racks + ? all_racks[i % num_broker_racks]->str + : NULL); + } +} + +/* Helper to populate the deduplicated racks inside each partition. It's assumed + * that `mdi->brokers` is set, maybe using + * `ut_populate_internal_broker_metadata`. */ +void ut_populate_internal_topic_metadata(rd_kafka_metadata_internal_t *mdi) { + int ti; + rd_kafka_metadata_broker_internal_t *brokers_internal; + size_t broker_cnt; + + rd_assert(mdi->brokers); + + brokers_internal = mdi->brokers; + broker_cnt = mdi->metadata.broker_cnt; + + for (ti = 0; ti < mdi->metadata.topic_cnt; ti++) { + int i; + rd_kafka_metadata_topic_t *mdt = &mdi->metadata.topics[ti]; + rd_kafka_metadata_topic_internal_t *mdti = &mdi->topics[ti]; + + for (i = 0; i < mdt->partition_cnt; i++) { + int j; + rd_kafka_metadata_partition_t *partition = + &mdt->partitions[i]; + rd_kafka_metadata_partition_internal_t + *partition_internal = &mdti->partitions[i]; + + rd_list_t *curr_list; + char *rack; + + if (partition->replica_cnt == 0) + continue; + + curr_list = rd_list_new( + 0, NULL); /* use a list for de-duplication */ + for (j = 0; j < partition->replica_cnt; j++) { + rd_kafka_metadata_broker_internal_t key = { + .id = partition->replicas[j]}; + rd_kafka_metadata_broker_internal_t *broker = + bsearch( + &key, brokers_internal, broker_cnt, + sizeof( + rd_kafka_metadata_broker_internal_t), + rd_kafka_metadata_broker_internal_cmp); + if (!broker || !broker->rack_id) + continue; + rd_list_add(curr_list, broker->rack_id); + } + rd_list_deduplicate(&curr_list, rd_strcmp2); + + partition_internal->racks_cnt = rd_list_cnt(curr_list); + partition_internal->racks = rd_malloc( + sizeof(char *) * partition_internal->racks_cnt); + RD_LIST_FOREACH(rack, curr_list, j) { + partition_internal->racks[j] = + rack; /* no duplication */ + } + rd_list_destroy(curr_list); + } + } +} + +/* Helper to destroy test metadata. Destroying the metadata has some additional + * steps in case of tests. */ +void ut_destroy_metadata(rd_kafka_metadata_t *md) { + int ti; + rd_kafka_metadata_internal_t *mdi = rd_kafka_metadata_get_internal(md); + + for (ti = 0; ti < md->topic_cnt; ti++) { + int i; + rd_kafka_metadata_topic_t *mdt = &md->topics[ti]; + rd_kafka_metadata_topic_internal_t *mdti = &mdi->topics[ti]; + + for (i = 0; mdti && i < mdt->partition_cnt; i++) { + rd_free(mdti->partitions[i].racks); + } + } + + rd_kafka_metadata_destroy(md); +} + + +/** + * @brief Set a member's owned partitions based on its assignment. + * + * For use between assignor_run(). This is mimicing a consumer receiving + * its new assignment and including it in the next rebalance as its + * owned-partitions. + */ +void ut_set_owned(rd_kafka_group_member_t *rkgm) { + if (rkgm->rkgm_owned) + rd_kafka_topic_partition_list_destroy(rkgm->rkgm_owned); + + rkgm->rkgm_owned = + rd_kafka_topic_partition_list_copy(rkgm->rkgm_assignment); +} + + +void ut_print_toppar_list(const rd_kafka_topic_partition_list_t *partitions) { + int i; + + for (i = 0; i < partitions->cnt; i++) + RD_UT_SAY(" %s [%" PRId32 "]", partitions->elems[i].topic, + partitions->elems[i].partition); +} + + +/* Implementation for ut_init_member and ut_init_member_with_rackv. */ +static void ut_init_member_internal(rd_kafka_group_member_t *rkgm, + const char *member_id, + const rd_kafkap_str_t *rack_id, + va_list ap) { + const char *topic; + + memset(rkgm, 0, sizeof(*rkgm)); + + rkgm->rkgm_member_id = rd_kafkap_str_new(member_id, -1); + rkgm->rkgm_group_instance_id = rd_kafkap_str_new(member_id, -1); + rkgm->rkgm_rack_id = rack_id ? rd_kafkap_str_copy(rack_id) : NULL; + + rd_list_init(&rkgm->rkgm_eligible, 0, NULL); + + rkgm->rkgm_subscription = rd_kafka_topic_partition_list_new(4); + + while ((topic = va_arg(ap, const char *))) + rd_kafka_topic_partition_list_add(rkgm->rkgm_subscription, + topic, RD_KAFKA_PARTITION_UA); + + rkgm->rkgm_assignment = + rd_kafka_topic_partition_list_new(rkgm->rkgm_subscription->size); + + rkgm->rkgm_generation = 1; +} + +/** + * @brief Initialize group member struct for testing. + * + * va-args is a NULL-terminated list of (const char *) topics. + * + * Use rd_kafka_group_member_clear() to free fields. + */ +void ut_init_member(rd_kafka_group_member_t *rkgm, const char *member_id, ...) { + va_list ap; + va_start(ap, member_id); + ut_init_member_internal(rkgm, member_id, NULL, ap); + va_end(ap); +} + +/** + * @brief Initialize group member struct for testing with a rackid. + * + * va-args is a NULL-terminated list of (const char *) topics. + * + * Use rd_kafka_group_member_clear() to free fields. + */ +void ut_init_member_with_rackv(rd_kafka_group_member_t *rkgm, + const char *member_id, + const rd_kafkap_str_t *rack_id, + ...) { + va_list ap; + va_start(ap, rack_id); + ut_init_member_internal(rkgm, member_id, rack_id, ap); + va_end(ap); +} + +/** + * @brief Initialize group member struct for testing with a rackid. + * + * Topics that the member is subscribed to are specified in an array with the + * size specified separately. + * + * Use rd_kafka_group_member_clear() to free fields. + */ +void ut_init_member_with_rack(rd_kafka_group_member_t *rkgm, + const char *member_id, + const rd_kafkap_str_t *rack_id, + char *topics[], + size_t topic_cnt) { + size_t i; + + memset(rkgm, 0, sizeof(*rkgm)); + + rkgm->rkgm_member_id = rd_kafkap_str_new(member_id, -1); + rkgm->rkgm_group_instance_id = rd_kafkap_str_new(member_id, -1); + rkgm->rkgm_rack_id = rack_id ? rd_kafkap_str_copy(rack_id) : NULL; + rd_list_init(&rkgm->rkgm_eligible, 0, NULL); + + rkgm->rkgm_subscription = rd_kafka_topic_partition_list_new(4); + + for (i = 0; i < topic_cnt; i++) { + rd_kafka_topic_partition_list_add( + rkgm->rkgm_subscription, topics[i], RD_KAFKA_PARTITION_UA); + } + rkgm->rkgm_assignment = + rd_kafka_topic_partition_list_new(rkgm->rkgm_subscription->size); +} + +/** + * @brief Verify that member's assignment matches the expected partitions. + * + * The va-list is a NULL-terminated list of (const char *topic, int partition) + * tuples. + * + * @returns 0 on success, else raises a unittest error and returns 1. + */ +int verifyAssignment0(const char *function, + int line, + rd_kafka_group_member_t *rkgm, + ...) { + va_list ap; + int cnt = 0; + const char *topic; + int fails = 0; + + va_start(ap, rkgm); + while ((topic = va_arg(ap, const char *))) { + int partition = va_arg(ap, int); + cnt++; + + if (!rd_kafka_topic_partition_list_find(rkgm->rkgm_assignment, + topic, partition)) { + RD_UT_WARN( + "%s:%d: Expected %s [%d] not found in %s's " + "assignment (%d partition(s))", + function, line, topic, partition, + rkgm->rkgm_member_id->str, + rkgm->rkgm_assignment->cnt); + fails++; + } + } + va_end(ap); + + if (cnt != rkgm->rkgm_assignment->cnt) { + RD_UT_WARN( + "%s:%d: " + "Expected %d assigned partition(s) for %s, not %d", + function, line, cnt, rkgm->rkgm_member_id->str, + rkgm->rkgm_assignment->cnt); + fails++; + } + + if (fails) + ut_print_toppar_list(rkgm->rkgm_assignment); + + RD_UT_ASSERT(!fails, "%s:%d: See previous errors", function, line); + + return 0; +} + +/** + * @brief Verify that all members' assignment matches the expected partitions. + * + * The va-list is a list of (const char *topic, int partition) + * tuples, and NULL to demarcate different members' assignment. + * + * @returns 0 on success, else raises a unittest error and returns 1. + */ +int verifyMultipleAssignment0(const char *function, + int line, + rd_kafka_group_member_t *rkgms, + size_t member_cnt, + ...) { + va_list ap; + const char *topic; + int fails = 0; + size_t i = 0; + + if (member_cnt == 0) { + return 0; + } + + va_start(ap, member_cnt); + for (i = 0; i < member_cnt; i++) { + rd_kafka_group_member_t *rkgm = &rkgms[i]; + int cnt = 0; + int local_fails = 0; + + while ((topic = va_arg(ap, const char *))) { + int partition = va_arg(ap, int); + cnt++; + + if (!rd_kafka_topic_partition_list_find( + rkgm->rkgm_assignment, topic, partition)) { + RD_UT_WARN( + "%s:%d: Expected %s [%d] not found in %s's " + "assignment (%d partition(s))", + function, line, topic, partition, + rkgm->rkgm_member_id->str, + rkgm->rkgm_assignment->cnt); + local_fails++; + } + } + + if (cnt != rkgm->rkgm_assignment->cnt) { + RD_UT_WARN( + "%s:%d: " + "Expected %d assigned partition(s) for %s, not %d", + function, line, cnt, rkgm->rkgm_member_id->str, + rkgm->rkgm_assignment->cnt); + fails++; + } + + if (local_fails) + ut_print_toppar_list(rkgm->rkgm_assignment); + fails += local_fails; + } + va_end(ap); + + RD_UT_ASSERT(!fails, "%s:%d: See previous errors", function, line); + + return 0; +} + + +#define verifyNumPartitionsWithRackMismatchPartition(rktpar, metadata, \ + increase) \ + do { \ + if (!rktpar) \ + break; \ + int i; \ + rd_bool_t noneMatch = rd_true; \ + rd_kafka_metadata_internal_t *metadata_internal = \ + rd_kafka_metadata_get_internal(metadata); \ + \ + for (i = 0; i < metadata->topics[j].partitions[k].replica_cnt; \ + i++) { \ + int32_t replica_id = \ + metadata->topics[j].partitions[k].replicas[i]; \ + rd_kafka_metadata_broker_internal_t *broker; \ + rd_kafka_metadata_broker_internal_find( \ + metadata_internal, replica_id, broker); \ + \ + if (broker && !strcmp(rack_id, broker->rack_id)) { \ + noneMatch = rd_false; \ + break; \ + } \ + } \ + \ + if (noneMatch) \ + increase++; \ + } while (0); + +/** + * @brief Verify number of partitions with rack mismatch. + */ +int verifyNumPartitionsWithRackMismatch0(const char *function, + int line, + rd_kafka_metadata_t *metadata, + rd_kafka_group_member_t *rkgms, + size_t member_cnt, + int expectedNumMismatch) { + size_t i; + int j, k; + + int numMismatched = 0; + for (i = 0; i < member_cnt; i++) { + rd_kafka_group_member_t *rkgm = &rkgms[i]; + const char *rack_id = rkgm->rkgm_rack_id->str; + if (rack_id) { + for (j = 0; j < metadata->topic_cnt; j++) { + for (k = 0; + k < metadata->topics[j].partition_cnt; + k++) { + rd_kafka_topic_partition_t *rktpar = + rd_kafka_topic_partition_list_find( + rkgm->rkgm_assignment, + metadata->topics[j].topic, k); + verifyNumPartitionsWithRackMismatchPartition( + rktpar, metadata, numMismatched); + } + } + } + } + + RD_UT_ASSERT(expectedNumMismatch == numMismatched, + "%s:%d: Expected %d mismatches, got %d", function, line, + expectedNumMismatch, numMismatched); + + return 0; +} + + +int verifyValidityAndBalance0(const char *func, + int line, + rd_kafka_group_member_t *members, + size_t member_cnt, + const rd_kafka_metadata_t *metadata) { + int fails = 0; + int i; + rd_bool_t verbose = rd_false; /* Enable for troubleshooting */ + + RD_UT_SAY("%s:%d: verifying assignment for %d member(s):", func, line, + (int)member_cnt); + + for (i = 0; i < (int)member_cnt; i++) { + const char *consumer = members[i].rkgm_member_id->str; + const rd_kafka_topic_partition_list_t *partitions = + members[i].rkgm_assignment; + int p, j; + + if (verbose) + RD_UT_SAY( + "%s:%d: " + "consumer \"%s\", %d subscribed topic(s), " + "%d assigned partition(s):", + func, line, consumer, + members[i].rkgm_subscription->cnt, partitions->cnt); + + for (p = 0; p < partitions->cnt; p++) { + const rd_kafka_topic_partition_t *partition = + &partitions->elems[p]; + + if (verbose) + RD_UT_SAY("%s:%d: %s [%" PRId32 "]", func, + line, partition->topic, + partition->partition); + + if (!rd_kafka_topic_partition_list_find( + members[i].rkgm_subscription, partition->topic, + RD_KAFKA_PARTITION_UA)) { + RD_UT_WARN("%s [%" PRId32 + "] is assigned to " + "%s but it is not subscribed to " + "that topic", + partition->topic, + partition->partition, consumer); + fails++; + } + } + + /* Update the member's owned partitions to match + * the assignment. */ + ut_set_owned(&members[i]); + + if (i == (int)member_cnt - 1) + continue; + + for (j = i + 1; j < (int)member_cnt; j++) { + const char *otherConsumer = + members[j].rkgm_member_id->str; + const rd_kafka_topic_partition_list_t *otherPartitions = + members[j].rkgm_assignment; + rd_bool_t balanced = + abs(partitions->cnt - otherPartitions->cnt) <= 1; + + for (p = 0; p < partitions->cnt; p++) { + const rd_kafka_topic_partition_t *partition = + &partitions->elems[p]; + + if (rd_kafka_topic_partition_list_find( + otherPartitions, partition->topic, + partition->partition)) { + RD_UT_WARN( + "Consumer %s and %s are both " + "assigned %s [%" PRId32 "]", + consumer, otherConsumer, + partition->topic, + partition->partition); + fails++; + } + + + /* If assignment is imbalanced and this topic + * is also subscribed by the other consumer + * it means the assignment strategy failed to + * properly balance the partitions. */ + if (!balanced && + rd_kafka_topic_partition_list_find_topic( + otherPartitions, partition->topic)) { + RD_UT_WARN( + "Some %s partition(s) can be " + "moved from " + "%s (%d partition(s)) to " + "%s (%d partition(s)) to " + "achieve a better balance", + partition->topic, consumer, + partitions->cnt, otherConsumer, + otherPartitions->cnt); + fails++; + } + } + } + } + + RD_UT_ASSERT(!fails, "%s:%d: See %d previous errors", func, line, + fails); + + return 0; +} + +/** + * @brief Checks that all assigned partitions are fully balanced. + * + * Only works for symmetrical subscriptions. + */ +int isFullyBalanced0(const char *function, + int line, + const rd_kafka_group_member_t *members, + size_t member_cnt) { + int min_assignment = INT_MAX; + int max_assignment = -1; + size_t i; + + for (i = 0; i < member_cnt; i++) { + int size = members[i].rkgm_assignment->cnt; + if (size < min_assignment) + min_assignment = size; + if (size > max_assignment) + max_assignment = size; + } + + RD_UT_ASSERT(max_assignment - min_assignment <= 1, + "%s:%d: Assignment not balanced: min %d, max %d", function, + line, min_assignment, max_assignment); + + return 0; +} /** @@ -881,6 +1572,7 @@ static int ut_assignors(void) { /* Run through test cases */ for (i = 0; tests[i].name; i++) { int ie, it, im; + rd_kafka_metadata_internal_t metadata_internal; rd_kafka_metadata_t metadata; rd_kafka_group_member_t *members; @@ -888,14 +1580,38 @@ static int ut_assignors(void) { metadata.topic_cnt = tests[i].topic_cnt; metadata.topics = rd_alloca(sizeof(*metadata.topics) * metadata.topic_cnt); + metadata_internal.topics = rd_alloca( + sizeof(*metadata_internal.topics) * metadata.topic_cnt); + memset(metadata.topics, 0, sizeof(*metadata.topics) * metadata.topic_cnt); + memset(metadata_internal.topics, 0, + sizeof(*metadata_internal.topics) * metadata.topic_cnt); + for (it = 0; it < metadata.topic_cnt; it++) { + int pt; metadata.topics[it].topic = (char *)tests[i].topics[it].name; metadata.topics[it].partition_cnt = tests[i].topics[it].partition_cnt; - metadata.topics[it].partitions = NULL; /* Not used */ + metadata.topics[it].partitions = + rd_alloca(metadata.topics[it].partition_cnt * + sizeof(rd_kafka_metadata_partition_t)); + metadata_internal.topics[it].partitions = rd_alloca( + metadata.topics[it].partition_cnt * + sizeof(rd_kafka_metadata_partition_internal_t)); + for (pt = 0; pt < metadata.topics[it].partition_cnt; + pt++) { + metadata.topics[it].partitions[pt].id = pt; + metadata.topics[it].partitions[pt].replica_cnt = + 0; + metadata_internal.topics[it] + .partitions[pt] + .racks_cnt = 0; + metadata_internal.topics[it] + .partitions[pt] + .racks = NULL; + } } /* Create members */ @@ -946,9 +1662,12 @@ static int ut_assignors(void) { } /* Run assignor */ - err = rd_kafka_assignor_run( - rk->rk_cgrp, rkas, &metadata, members, - tests[i].member_cnt, errstr, sizeof(errstr)); + metadata_internal.metadata = metadata; + err = rd_kafka_assignor_run( + rk->rk_cgrp, rkas, + (rd_kafka_metadata_t *)(&metadata_internal), + members, tests[i].member_cnt, errstr, + sizeof(errstr)); RD_UT_ASSERT(!err, "Assignor case %s for %s failed: %s", tests[i].name, diff --git a/src/rdkafka_assignor.h b/src/rdkafka_assignor.h index b90e7dc980..9d01d38065 100644 --- a/src/rdkafka_assignor.h +++ b/src/rdkafka_assignor.h @@ -28,7 +28,7 @@ #ifndef _RDKAFKA_ASSIGNOR_H_ #define _RDKAFKA_ASSIGNOR_H_ - +#include "rdkafka_metadata.h" /*! * Enumerates the different rebalance protocol types. @@ -69,6 +69,8 @@ typedef struct rd_kafka_group_member_s { rd_kafkap_bytes_t *rkgm_member_metadata; /** Group generation id. */ int rkgm_generation; + /** Member rack id. */ + rd_kafkap_str_t *rkgm_rack_id; } rd_kafka_group_member_t; @@ -78,13 +80,13 @@ int rd_kafka_group_member_find_subscription(rd_kafka_t *rk, const rd_kafka_group_member_t *rkgm, const char *topic); - /** * Structure to hold metadata for a single topic and all its * subscribing members. */ typedef struct rd_kafka_assignor_topic_s { const rd_kafka_metadata_topic_t *metadata; + const rd_kafka_metadata_topic_internal_t *metadata_internal; rd_list_t members; /* rd_kafka_group_member_t * */ } rd_kafka_assignor_topic_t; @@ -120,7 +122,8 @@ typedef struct rd_kafka_assignor_s { const struct rd_kafka_assignor_s *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions); + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id); void (*rkas_on_assignment_cb)( const struct rd_kafka_assignor_s *rkas, @@ -158,7 +161,8 @@ rd_kafka_resp_err_t rd_kafka_assignor_add( const struct rd_kafka_assignor_s *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions), + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id), void (*on_assignment_cb)(const struct rd_kafka_assignor_s *rkas, void **assignor_state, const rd_kafka_topic_partition_list_t *assignment, @@ -172,13 +176,16 @@ rd_kafkap_bytes_t *rd_kafka_consumer_protocol_member_metadata_new( const rd_list_t *topics, const void *userdata, size_t userdata_size, - const rd_kafka_topic_partition_list_t *owned_partitions); + const rd_kafka_topic_partition_list_t *owned_partitions, + int generation, + const rd_kafkap_str_t *rack_id); rd_kafkap_bytes_t *rd_kafka_assignor_get_metadata_with_empty_userdata( const rd_kafka_assignor_t *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions); + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id); void rd_kafka_assignor_update_subscription( @@ -208,5 +215,187 @@ void rd_kafka_group_member_clear(rd_kafka_group_member_t *rkgm); rd_kafka_resp_err_t rd_kafka_range_assignor_init(rd_kafka_t *rk); rd_kafka_resp_err_t rd_kafka_roundrobin_assignor_init(rd_kafka_t *rk); rd_kafka_resp_err_t rd_kafka_sticky_assignor_init(rd_kafka_t *rk); +rd_bool_t +rd_kafka_use_rack_aware_assignment(rd_kafka_assignor_topic_t **topics, + size_t topic_cnt, + const rd_kafka_metadata_internal_t *mdi); + +/** + * @name Common unit test functions, macros, and enums to use across assignors. + * + * + * + */ + +/* Tests can be parametrized to contain either only broker racks, only consumer + * racks or both.*/ +typedef enum { + RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK = 0, + RD_KAFKA_RANGE_ASSIGNOR_UT_NO_CONSUMER_RACK = 1, + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK = 2, + RD_KAFKA_RANGE_ASSIGNOR_UT_CONFIG_CNT = 3, +} rd_kafka_assignor_ut_rack_config_t; + + +void ut_populate_internal_broker_metadata(rd_kafka_metadata_internal_t *mdi, + int num_broker_racks, + rd_kafkap_str_t *all_racks[], + size_t all_racks_cnt); + +void ut_populate_internal_topic_metadata(rd_kafka_metadata_internal_t *mdi); + +void ut_destroy_metadata(rd_kafka_metadata_t *md); + +void ut_set_owned(rd_kafka_group_member_t *rkgm); + +void ut_print_toppar_list(const rd_kafka_topic_partition_list_t *partitions); + +void ut_init_member(rd_kafka_group_member_t *rkgm, const char *member_id, ...); + +void ut_init_member_with_rackv(rd_kafka_group_member_t *rkgm, + const char *member_id, + const rd_kafkap_str_t *rack_id, + ...); + +void ut_init_member_with_rack(rd_kafka_group_member_t *rkgm, + const char *member_id, + const rd_kafkap_str_t *rack_id, + char *topics[], + size_t topic_cnt); + +int verifyAssignment0(const char *function, + int line, + rd_kafka_group_member_t *rkgm, + ...); + +int verifyMultipleAssignment0(const char *function, + int line, + rd_kafka_group_member_t *rkgms, + size_t member_cnt, + ...); + +int verifyNumPartitionsWithRackMismatch0(const char *function, + int line, + rd_kafka_metadata_t *metadata, + rd_kafka_group_member_t *rkgms, + size_t member_cnt, + int expectedNumMismatch); + +#define verifyAssignment(rkgm, ...) \ + do { \ + if (verifyAssignment0(__FUNCTION__, __LINE__, rkgm, \ + __VA_ARGS__)) \ + return 1; \ + } while (0) + +#define verifyMultipleAssignment(rkgms, member_cnt, ...) \ + do { \ + if (verifyMultipleAssignment0(__FUNCTION__, __LINE__, rkgms, \ + member_cnt, __VA_ARGS__)) \ + return 1; \ + } while (0) + +#define verifyNumPartitionsWithRackMismatch(metadata, rkgms, member_cnt, \ + expectedNumMismatch) \ + do { \ + if (verifyNumPartitionsWithRackMismatch0( \ + __FUNCTION__, __LINE__, metadata, rkgms, member_cnt, \ + expectedNumMismatch)) \ + return 1; \ + } while (0) + +int verifyValidityAndBalance0(const char *func, + int line, + rd_kafka_group_member_t *members, + size_t member_cnt, + const rd_kafka_metadata_t *metadata); + +#define verifyValidityAndBalance(members, member_cnt, metadata) \ + do { \ + if (verifyValidityAndBalance0(__FUNCTION__, __LINE__, members, \ + member_cnt, metadata)) \ + return 1; \ + } while (0) + +int isFullyBalanced0(const char *function, + int line, + const rd_kafka_group_member_t *members, + size_t member_cnt); + +#define isFullyBalanced(members, member_cnt) \ + do { \ + if (isFullyBalanced0(__FUNCTION__, __LINE__, members, \ + member_cnt)) \ + return 1; \ + } while (0) + +/* Helper macro to initialize a consumer with or without a rack depending on the + * value of parametrization. */ +#define ut_initMemberConditionalRack(member_ptr, member_id, rack, \ + parametrization, ...) \ + do { \ + if (parametrization == \ + RD_KAFKA_RANGE_ASSIGNOR_UT_NO_CONSUMER_RACK) { \ + ut_init_member(member_ptr, member_id, __VA_ARGS__); \ + } else { \ + ut_init_member_with_rackv(member_ptr, member_id, rack, \ + __VA_ARGS__); \ + } \ + } while (0) + +/* Helper macro to initialize rd_kafka_metadata_t* with or without replicas + * depending on the value of parametrization. This accepts variadic arguments + * for topics. */ +#define ut_initMetadataConditionalRack(metadataPtr, replication_factor, \ + num_broker_racks, all_racks, \ + all_racks_cnt, parametrization, ...) \ + do { \ + int num_brokers = num_broker_racks > 0 \ + ? replication_factor * num_broker_racks \ + : replication_factor; \ + if (parametrization == \ + RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { \ + *(metadataPtr) = \ + rd_kafka_metadata_new_topic_mockv(__VA_ARGS__); \ + } else { \ + *(metadataPtr) = \ + rd_kafka_metadata_new_topic_with_partition_replicas_mockv( \ + replication_factor, num_brokers, __VA_ARGS__); \ + ut_populate_internal_broker_metadata( \ + rd_kafka_metadata_get_internal(*(metadataPtr)), \ + num_broker_racks, all_racks, all_racks_cnt); \ + ut_populate_internal_topic_metadata( \ + rd_kafka_metadata_get_internal(*(metadataPtr))); \ + } \ + } while (0) + + +/* Helper macro to initialize rd_kafka_metadata_t* with or without replicas + * depending on the value of parametrization. This accepts a list of topics, + * rather than being variadic. + */ +#define ut_initMetadataConditionalRack0( \ + metadataPtr, replication_factor, num_broker_racks, all_racks, \ + all_racks_cnt, parametrization, topics, topic_cnt) \ + do { \ + int num_brokers = num_broker_racks > 0 \ + ? replication_factor * num_broker_racks \ + : replication_factor; \ + if (parametrization == \ + RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { \ + *(metadataPtr) = rd_kafka_metadata_new_topic_mock( \ + topics, topic_cnt, -1, 0); \ + } else { \ + *(metadataPtr) = rd_kafka_metadata_new_topic_mock( \ + topics, topic_cnt, replication_factor, \ + num_brokers); \ + ut_populate_internal_broker_metadata( \ + rd_kafka_metadata_get_internal(*(metadataPtr)), \ + num_broker_racks, all_racks, all_racks_cnt); \ + ut_populate_internal_topic_metadata( \ + rd_kafka_metadata_get_internal(*(metadataPtr))); \ + } \ + } while (0) + #endif /* _RDKAFKA_ASSIGNOR_H_ */ diff --git a/src/rdkafka_aux.c b/src/rdkafka_aux.c index 753f03d678..da565d1594 100644 --- a/src/rdkafka_aux.c +++ b/src/rdkafka_aux.c @@ -234,7 +234,7 @@ void rd_kafka_acl_result_free(void *ptr) { * @return A new allocated Node object. * Use rd_kafka_Node_destroy() to free when done. */ -rd_kafka_Node_t *rd_kafka_Node_new(int id, +rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, const char *host, uint16_t port, const char *rack_id) { diff --git a/src/rdkafka_aux.h b/src/rdkafka_aux.h index ccf18e91e7..7d5339bd73 100644 --- a/src/rdkafka_aux.h +++ b/src/rdkafka_aux.h @@ -110,8 +110,10 @@ typedef struct rd_kafka_Node_s { char *rack_id; /*< (optional) Node rack id */ } rd_kafka_Node_t; -rd_kafka_Node_t * -rd_kafka_Node_new(int id, const char *host, uint16_t port, const char *rack_id); +rd_kafka_Node_t *rd_kafka_Node_new(int32_t id, + const char *host, + uint16_t port, + const char *rack_id); rd_kafka_Node_t *rd_kafka_Node_copy(const rd_kafka_Node_t *src); diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h index b4f606317b..9cb30a0df8 100644 --- a/src/rdkafka_buf.h +++ b/src/rdkafka_buf.h @@ -366,6 +366,9 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */ rd_bool_t all_topics; /**< Full/All topics requested */ rd_bool_t cgrp_update; /**< Update cgrp with topic * status from response. */ + rd_bool_t force_racks; /**< Force the returned metadata + * to contain partition to + * rack mapping. */ int *decr; /* Decrement this integer by one * when request is complete: @@ -682,6 +685,10 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */ size_t _slen; \ char *_dst; \ rd_kafka_buf_read_str(rkbuf, &_kstr); \ + if (RD_KAFKAP_STR_IS_NULL(&_kstr)) { \ + dst = NULL; \ + break; \ + } \ _slen = RD_KAFKAP_STR_LEN(&_kstr); \ if (!(_dst = rd_tmpabuf_write(tmpabuf, _kstr.str, _slen + 1))) \ rd_kafka_buf_parse_fail( \ @@ -784,9 +791,8 @@ struct rd_kafka_buf_s { /* rd_kafka_buf_t */ uint64_t _tagtype, _taglen; \ rd_kafka_buf_read_uvarint(rkbuf, &_tagtype); \ rd_kafka_buf_read_uvarint(rkbuf, &_taglen); \ - if (_taglen > 1) \ - rd_kafka_buf_skip(rkbuf, \ - (size_t)(_taglen - 1)); \ + if (_taglen > 0) \ + rd_kafka_buf_skip(rkbuf, (size_t)(_taglen)); \ } \ } while (0) diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 43d012106c..61cbd1f16f 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -415,7 +415,7 @@ rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk, rkcg->rkcg_wait_coord_q = rd_kafka_q_new(rk); rkcg->rkcg_wait_coord_q->rkq_serve = rkcg->rkcg_ops->rkq_serve; rkcg->rkcg_wait_coord_q->rkq_opaque = rkcg->rkcg_ops->rkq_opaque; - rkcg->rkcg_q = rd_kafka_q_new(rk); + rkcg->rkcg_q = rd_kafka_consume_q_new(rk); rkcg->rkcg_group_instance_id = rd_kafkap_str_new(rk->rk_conf.group_instance_id, -1); @@ -1662,7 +1662,7 @@ static void rd_kafka_cgrp_handle_SyncGroup(rd_kafka_t *rk, static void rd_kafka_cgrp_assignor_run(rd_kafka_cgrp_t *rkcg, rd_kafka_assignor_t *rkas, rd_kafka_resp_err_t err, - rd_kafka_metadata_t *metadata, + rd_kafka_metadata_internal_t *metadata, rd_kafka_group_member_t *members, int member_cnt) { char errstr[512]; @@ -1677,8 +1677,8 @@ static void rd_kafka_cgrp_assignor_run(rd_kafka_cgrp_t *rkcg, *errstr = '\0'; /* Run assignor */ - err = rd_kafka_assignor_run(rkcg, rkas, metadata, members, member_cnt, - errstr, sizeof(errstr)); + err = rd_kafka_assignor_run(rkcg, rkas, &metadata->metadata, members, + member_cnt, errstr, sizeof(errstr)); if (err) { if (!*errstr) @@ -1745,7 +1745,7 @@ rd_kafka_cgrp_assignor_handle_Metadata_op(rd_kafka_t *rk, } rd_kafka_cgrp_assignor_run(rkcg, rkcg->rkcg_assignor, rko->rko_err, - rko->rko_u.metadata.md, + rko->rko_u.metadata.mdi, rkcg->rkcg_group_leader.members, rkcg->rkcg_group_leader.member_cnt); @@ -1777,9 +1777,12 @@ static int rd_kafka_group_MemberMetadata_consumer_read( rkbuf = rd_kafka_buf_new_shadow( MemberMetadata->data, RD_KAFKAP_BYTES_LEN(MemberMetadata), NULL); - /* Protocol parser needs a broker handle to log errors on. */ - rkbuf->rkbuf_rkb = rkb; - rd_kafka_broker_keep(rkb); + /* Protocol parser needs a broker handle to log errors on. + * If none is provided, don't log errors (mainly for unit tests). */ + if (rkb) { + rkbuf->rkbuf_rkb = rkb; + rd_kafka_broker_keep(rkb); + } rd_kafka_buf_read_i16(rkbuf, &Version); rd_kafka_buf_read_i32(rkbuf, &subscription_cnt); @@ -1810,6 +1813,16 @@ static int rd_kafka_group_MemberMetadata_consumer_read( rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields))) goto err; + if (Version >= 2) { + rd_kafka_buf_read_i32(rkbuf, &rkgm->rkgm_generation); + } + + if (Version >= 3) { + rd_kafkap_str_t RackId = RD_KAFKAP_STR_INITIALIZER; + rd_kafka_buf_read_str(rkbuf, &RackId); + rkgm->rkgm_rack_id = rd_kafkap_str_copy(&RackId); + } + rd_kafka_buf_destroy(rkbuf); return 0; @@ -1818,10 +1831,11 @@ static int rd_kafka_group_MemberMetadata_consumer_read( err = rkbuf->rkbuf_err; err: - rd_rkb_dbg(rkb, CGRP, "MEMBERMETA", - "Failed to parse MemberMetadata for \"%.*s\": %s", - RD_KAFKAP_STR_PR(rkgm->rkgm_member_id), - rd_kafka_err2str(err)); + if (rkb) + rd_rkb_dbg(rkb, CGRP, "MEMBERMETA", + "Failed to parse MemberMetadata for \"%.*s\": %s", + RD_KAFKAP_STR_PR(rkgm->rkgm_member_id), + rd_kafka_err2str(err)); if (rkgm->rkgm_subscription) { rd_kafka_topic_partition_list_destroy(rkgm->rkgm_subscription); rkgm->rkgm_subscription = NULL; @@ -1950,6 +1964,7 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, int sub_cnt = 0; rd_list_t topics; rd_kafka_op_t *rko; + rd_bool_t any_member_rack = rd_false; rd_kafka_dbg(rkb->rkb_rk, CGRP, "JOINGROUP", "I am elected leader for group \"%s\" " "with %" PRId32 " member(s)", @@ -1995,6 +2010,9 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, rd_kafka_topic_partition_list_get_topic_names( rkgm->rkgm_subscription, &topics, 0 /*dont include regex*/); + if (!any_member_rack && rkgm->rkgm_rack_id && + RD_KAFKAP_STR_LEN(rkgm->rkgm_rack_id)) + any_member_rack = rd_true; } } @@ -2024,8 +2042,8 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, rd_kafka_MetadataRequest( rkb, &topics, "partition assignor", rd_false /*!allow_auto_create*/, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, /* cgrp_update=false: * Since the subscription list may not be identical * across all members of the group and thus the @@ -2034,7 +2052,11 @@ static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk, * avoid triggering a rejoin or error propagation * on receiving the response since some topics * may be missing. */ - rd_false, rko, NULL, 0, NULL); + rd_false, + /* force_racks is true if any memeber has a client rack set, + since we will require partition to rack mapping in that + case for rack-aware assignors. */ + any_member_rack, rko, NULL, 0, NULL); rd_list_destroy(&topics); } else { @@ -3111,7 +3133,8 @@ static void rd_kafka_cgrp_op_handle_OffsetCommit(rd_kafka_t *rk, !(err == RD_KAFKA_RESP_ERR__NO_OFFSET && rko_orig->rko_u.offset_commit.silent_empty)) { /* Propagate commit results (success or permanent error) - * unless we're shutting down or commit was empty. */ + * unless we're shutting down or commit was empty, or if + * there was a rebalance in progress. */ rd_kafka_cgrp_propagate_commit_result(rkcg, rko_orig, err, errcnt, offsets); } @@ -5956,6 +5979,75 @@ static int unittest_list_to_map(void) { RD_UT_PASS(); } +int unittest_member_metadata_serdes(void) { + rd_list_t *topics = rd_list_new(0, (void *)rd_kafka_topic_info_destroy); + rd_kafka_topic_partition_list_t *owned_partitions = + rd_kafka_topic_partition_list_new(0); + rd_kafkap_str_t *rack_id = rd_kafkap_str_new("myrack", -1); + const void *userdata = NULL; + const int32_t userdata_size = 0; + const int generation = 3; + const char topic_name[] = "mytopic"; + rd_kafka_group_member_t *rkgm; + int version; + + rd_list_add(topics, rd_kafka_topic_info_new(topic_name, 3)); + rd_kafka_topic_partition_list_add(owned_partitions, topic_name, 0); + rkgm = rd_calloc(1, sizeof(*rkgm)); + + /* Note that the version variable doesn't actually change the Version + * field in the serialized message. It only runs the tests with/without + * additional fields added in that particular version. */ + for (version = 0; version <= 3; version++) { + rd_kafkap_bytes_t *member_metadata; + + /* Serialize. */ + member_metadata = + rd_kafka_consumer_protocol_member_metadata_new( + topics, userdata, userdata_size, + version >= 1 ? owned_partitions : NULL, + version >= 2 ? generation : -1, + version >= 3 ? rack_id : NULL); + + /* Deserialize. */ + rd_kafka_group_MemberMetadata_consumer_read(NULL, rkgm, + member_metadata); + + /* Compare results. */ + RD_UT_ASSERT(rkgm->rkgm_subscription->cnt == + rd_list_cnt(topics), + "subscription size should be correct"); + RD_UT_ASSERT(!strcmp(topic_name, + rkgm->rkgm_subscription->elems[0].topic), + "subscriptions should be correct"); + RD_UT_ASSERT(rkgm->rkgm_userdata->len == userdata_size, + "userdata should have the size 0"); + if (version >= 1) + RD_UT_ASSERT(!rd_kafka_topic_partition_list_cmp( + rkgm->rkgm_owned, owned_partitions, + rd_kafka_topic_partition_cmp), + "owned partitions should be same"); + if (version >= 2) + RD_UT_ASSERT(generation == rkgm->rkgm_generation, + "generation should be same"); + if (version >= 3) + RD_UT_ASSERT( + !rd_kafkap_str_cmp(rack_id, rkgm->rkgm_rack_id), + "rack id should be same"); + + rd_kafka_group_member_clear(rkgm); + rd_kafkap_bytes_destroy(member_metadata); + } + + /* Clean up. */ + rd_list_destroy(topics); + rd_kafka_topic_partition_list_destroy(owned_partitions); + rd_kafkap_str_destroy(rack_id); + rd_free(rkgm); + + RD_UT_PASS(); +} + /** * @brief Consumer group unit tests @@ -5968,6 +6060,7 @@ int unittest_cgrp(void) { fails += unittest_set_subtract(); fails += unittest_map_to_list(); fails += unittest_list_to_map(); + fails += unittest_member_metadata_serdes(); return fails; } diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 8ee67a4205..648a55b93d 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -879,8 +879,6 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { * 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 { @@ -975,7 +973,25 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { return cnt; } - +/** + * @brief Decide whether it should start fetching from next fetch start + * or continue with current fetch pos. + * + * @param rktp the toppar + * + * @returns rd_true if it should start fetching from next fetch start, + * rd_false otherwise. + * + * @locality any + * @locks toppar_lock() MUST be held + */ +rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start( + rd_kafka_toppar_t *rktp) { + return rktp->rktp_op_version > rktp->rktp_fetch_version || + 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; +} /** * @brief Decide whether this toppar should be on the fetch list or not. @@ -1037,10 +1053,7 @@ 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 || - 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) { + if (rd_kafka_toppar_fetch_decide_start_from_next_fetch_start(rktp)) { /* New version barrier, something was modified from the * control plane. Reset and start over. * Alternatively only the next_offset changed but not the diff --git a/src/rdkafka_fetcher.h b/src/rdkafka_fetcher.h index 0e3af82bb2..c2235b0d0c 100644 --- a/src/rdkafka_fetcher.h +++ b/src/rdkafka_fetcher.h @@ -33,6 +33,9 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now); +rd_bool_t rd_kafka_toppar_fetch_decide_start_from_next_fetch_start( + rd_kafka_toppar_t *rktp); + rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, rd_kafka_broker_t *rkb, int force_remove); diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h index 584ff3c965..6da9ecd52b 100644 --- a/src/rdkafka_int.h +++ b/src/rdkafka_int.h @@ -78,7 +78,8 @@ struct rd_kafka_topic_s; struct rd_kafka_msg_s; struct rd_kafka_broker_s; struct rd_kafka_toppar_s; - +typedef struct rd_kafka_metadata_internal_s rd_kafka_metadata_internal_t; +typedef struct rd_kafka_toppar_s rd_kafka_toppar_t; typedef struct rd_kafka_lwtopic_s rd_kafka_lwtopic_t; @@ -350,8 +351,9 @@ struct rd_kafka_s { rd_ts_t rk_ts_metadata; /* Timestamp of most recent * metadata. */ - struct rd_kafka_metadata *rk_full_metadata; /* Last full metadata. */ - rd_ts_t rk_ts_full_metadata; /* Timesstamp of .. */ + rd_kafka_metadata_internal_t + *rk_full_metadata; /* Last full metadata. */ + rd_ts_t rk_ts_full_metadata; /* Timestamp of .. */ struct rd_kafka_metadata_cache rk_metadata_cache; /* Metadata cache */ char *rk_clusterid; /* ClusterId from metadata */ diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c index c204326d99..4550451997 100644 --- a/src/rdkafka_metadata.c +++ b/src/rdkafka_metadata.c @@ -38,6 +38,25 @@ #include #include +/** + * @brief Id comparator for rd_kafka_metadata_broker_internal_t + */ +int rd_kafka_metadata_broker_internal_cmp(const void *_a, const void *_b) { + const rd_kafka_metadata_broker_internal_t *a = _a; + const rd_kafka_metadata_broker_internal_t *b = _b; + return RD_CMP(a->id, b->id); +} + +/** + * @brief Id comparator for rd_kafka_metadata_partition_internal_t + */ +static int rd_kafka_metadata_partition_internal_cmp(const void *_a, + const void *_b) { + const rd_kafka_metadata_partition_internal_t *a = _a; + const rd_kafka_metadata_partition_internal_t *b = _b; + return RD_CMP(a->id, b->id); +} + rd_kafka_resp_err_t rd_kafka_metadata(rd_kafka_t *rk, @@ -84,15 +103,15 @@ rd_kafka_metadata(rd_kafka_t *rk, * of outstanding metadata requests. */ rd_kafka_MetadataRequest( rkb, &topics, "application requested", allow_auto_create_topics, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, /* cgrp_update: * Only update consumer group state * on response if this lists all * topics in the cluster, since a * partial request may make it seem * like some subscribed topics are missing. */ - all_topics ? rd_true : rd_false, rko, NULL, 0, NULL); + all_topics ? rd_true : rd_false, rd_false /* force_racks */, rko, NULL, 0, NULL); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); @@ -115,8 +134,9 @@ rd_kafka_metadata(rd_kafka_t *rk, /* Reply: pass metadata pointer to application who now owns it*/ rd_kafka_assert(rk, rko->rko_u.metadata.md); - *metadatap = rko->rko_u.metadata.md; - rko->rko_u.metadata.md = NULL; + *metadatap = rko->rko_u.metadata.md; + rko->rko_u.metadata.md = NULL; + rko->rko_u.metadata.mdi = NULL; rd_kafka_op_destroy(rko); return RD_KAFKA_RESP_ERR_NO_ERROR; @@ -129,12 +149,13 @@ void rd_kafka_metadata_destroy(const struct rd_kafka_metadata *metadata) { } -/** - * @returns a newly allocated copy of metadata \p src of size \p size - */ -struct rd_kafka_metadata * -rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { +static rd_kafka_metadata_internal_t *rd_kafka_metadata_copy_internal( + const rd_kafka_metadata_internal_t *src_internal, + size_t size, + rd_bool_t populate_racks) { struct rd_kafka_metadata *md; + rd_kafka_metadata_internal_t *mdi; + const struct rd_kafka_metadata *src = &src_internal->metadata; rd_tmpabuf_t tbuf; int i; @@ -145,23 +166,37 @@ rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { * any pointer fields needs to be copied explicitly to update * the pointer address. */ rd_tmpabuf_new(&tbuf, size, 1 /*assert on fail*/); - md = rd_tmpabuf_write(&tbuf, src, sizeof(*md)); + mdi = rd_tmpabuf_write(&tbuf, src, sizeof(*mdi)); + md = &mdi->metadata; rd_tmpabuf_write_str(&tbuf, src->orig_broker_name); /* Copy Brokers */ md->brokers = rd_tmpabuf_write(&tbuf, src->brokers, - md->broker_cnt * sizeof(*md->brokers)); + src->broker_cnt * sizeof(*src->brokers)); + /* Copy internal Brokers */ + mdi->brokers = + rd_tmpabuf_write(&tbuf, src_internal->brokers, + src->broker_cnt * sizeof(*src_internal->brokers)); - for (i = 0; i < md->broker_cnt; i++) + for (i = 0; i < md->broker_cnt; i++) { md->brokers[i].host = rd_tmpabuf_write_str(&tbuf, src->brokers[i].host); + if (src_internal->brokers[i].rack_id) { + mdi->brokers[i].rack_id = rd_tmpabuf_write_str( + &tbuf, src_internal->brokers[i].rack_id); + } + } /* Copy TopicMetadata */ md->topics = rd_tmpabuf_write(&tbuf, src->topics, md->topic_cnt * sizeof(*md->topics)); + /* Copy internal TopicMetadata */ + mdi->topics = + rd_tmpabuf_write(&tbuf, src_internal->topics, + md->topic_cnt * sizeof(*src_internal->topics)); for (i = 0; i < md->topic_cnt; i++) { int j; @@ -175,8 +210,17 @@ rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { rd_tmpabuf_write(&tbuf, src->topics[i].partitions, md->topics[i].partition_cnt * sizeof(*md->topics[i].partitions)); + /* Copy internal partitions */ + mdi->topics[i].partitions = rd_tmpabuf_write( + &tbuf, src_internal->topics[i].partitions, + md->topics[i].partition_cnt * + sizeof(*src_internal->topics[i].partitions)); for (j = 0; j < md->topics[i].partition_cnt; j++) { + int k; + char *rack; + rd_list_t *curr_list; + /* Copy replicas and ISRs */ md->topics[i].partitions[j].replicas = rd_tmpabuf_write( &tbuf, src->topics[i].partitions[j].replicas, @@ -187,6 +231,59 @@ rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { &tbuf, src->topics[i].partitions[j].isrs, md->topics[i].partitions[j].isr_cnt * sizeof(*md->topics[i].partitions[j].isrs)); + + mdi->topics[i].partitions[j].racks_cnt = 0; + mdi->topics[i].partitions[j].racks = NULL; + + /* Iterate through replicas and populate racks, if + * needed. */ + if (!populate_racks) + continue; + + /* This is quite possibly a recomputation, because we've + * already done this for the src_internal. However, + * since the racks need to point inside the tmpbuf, we + * make this calculation again. Since this is done only + * in a case of a full metadata refresh, this will be + * fairly rare. */ + curr_list = rd_list_new(0, NULL); + for (k = 0; k < md->topics[i].partitions[j].replica_cnt; + k++) { + rd_kafka_metadata_broker_internal_t key = { + .id = md->topics[i] + .partitions[j] + .replicas[k]}; + rd_kafka_metadata_broker_internal_t *found = + bsearch( + &key, mdi->brokers, md->broker_cnt, + sizeof( + rd_kafka_metadata_broker_internal_t), + rd_kafka_metadata_broker_internal_cmp); + if (!found || !found->rack_id) + continue; + rd_list_add(curr_list, found->rack_id); + } + + if (!rd_list_cnt(curr_list)) { + rd_list_destroy(curr_list); + continue; + } + + rd_list_deduplicate(&curr_list, rd_strcmp2); + + mdi->topics[i].partitions[j].racks_cnt = + rd_list_cnt(curr_list); + mdi->topics[i].partitions[j].racks = rd_tmpabuf_alloc( + &tbuf, sizeof(char *) * rd_list_cnt(curr_list)); + RD_LIST_FOREACH(rack, curr_list, k) { + /* We don't copy here,`rack` points to memory + * inside `mdi` already, and it's allocated + * within a tmpabuf. So, the lifetime of + * mdi->topics[i].partitions[j].racks[k] is the + * same as the lifetime of the outer `mdi`. */ + mdi->topics[i].partitions[j].racks[k] = rack; + } + rd_list_destroy(curr_list); } } @@ -194,30 +291,38 @@ rd_kafka_metadata_copy(const struct rd_kafka_metadata *src, size_t size) { if (rd_tmpabuf_failed(&tbuf)) rd_kafka_assert(NULL, !*"metadata copy failed"); - /* Delibarely not destroying the tmpabuf since we return + /* Deliberately not destroying the tmpabuf since we return * its allocated memory. */ - return md; + return mdi; } - /** - * @brief Partition (id) comparator for partition_id_leader_epoch struct. + * @returns a newly allocated copy of metadata \p src of size \p size */ -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); +rd_kafka_metadata_internal_t * +rd_kafka_metadata_copy(const rd_kafka_metadata_internal_t *src_internal, + size_t size) { + return rd_kafka_metadata_copy_internal(src_internal, size, rd_false); } +/** + * @returns a newly allocated copy of metadata \p src of size \p size, with + * partition racks included. + */ +rd_kafka_metadata_internal_t *rd_kafka_metadata_copy_add_racks( + const rd_kafka_metadata_internal_t *src_internal, + size_t size) { + return rd_kafka_metadata_copy_internal(src_internal, size, rd_true); +} /** * @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. + * @param mdit Topic internal metadata. * * @locality rdkafka main thread * @locks_acquired rd_kafka_wrlock(rk) @@ -225,7 +330,7 @@ static int rd_kafka_metadata_partition_leader_epoch_cmp(const void *_a, 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) { + const rd_kafka_metadata_topic_internal_t *mdit) { rd_rkb_dbg(rkb, METADATA, "METADATA", /* The indent below is intentional */ @@ -246,7 +351,7 @@ static void rd_kafka_parse_Metadata_update_topic( } else { /* Update local topic & partition state based * on metadata */ - rd_kafka_topic_metadata_update2(rkb, mdt, leader_epochs); + rd_kafka_topic_metadata_update2(rkb, mdt, mdit); } } @@ -270,39 +375,92 @@ rd_bool_t rd_kafka_has_reliable_leader_epochs(rd_kafka_broker_t *rkb) { return ApiVersion >= 9; } +/* Populates the topic partition to rack mapping for the the topic given by + * `topic_idx` in the `mdi`. It's assumed that the internal broker metadata is + * already populated. */ +static void +rd_kafka_populate_metadata_topic_racks(rd_tmpabuf_t *tbuf, + size_t topic_idx, + rd_kafka_metadata_internal_t *mdi) { + rd_kafka_metadata_broker_internal_t *brokers_internal; + size_t broker_cnt; + int i; + rd_kafka_metadata_topic_t *mdt; + rd_kafka_metadata_topic_internal_t *mdti; + + rd_dassert(mdi->brokers); + rd_dassert(mdi->metadata.topic_cnt > (int)topic_idx); + + brokers_internal = mdi->brokers; + broker_cnt = mdi->metadata.broker_cnt; + + mdt = &mdi->metadata.topics[topic_idx]; + mdti = &mdi->topics[topic_idx]; + + for (i = 0; i < mdt->partition_cnt; i++) { + int j; + rd_kafka_metadata_partition_t *mdp = &mdt->partitions[i]; + rd_kafka_metadata_partition_internal_t *mdpi = + &mdti->partitions[i]; + + rd_list_t *curr_list; + char *rack; + + if (mdp->replica_cnt == 0) + continue; + + curr_list = + rd_list_new(0, NULL); /* use a list for de-duplication */ + for (j = 0; j < mdp->replica_cnt; j++) { + rd_kafka_metadata_broker_internal_t key = { + .id = mdp->replicas[j]}; + rd_kafka_metadata_broker_internal_t *broker = + bsearch(&key, brokers_internal, broker_cnt, + sizeof(rd_kafka_metadata_broker_internal_t), + rd_kafka_metadata_broker_internal_cmp); + if (!broker || !broker->rack_id) + continue; + rd_list_add(curr_list, broker->rack_id); + } + rd_list_deduplicate(&curr_list, rd_strcmp2); + + mdpi->racks_cnt = rd_list_cnt(curr_list); + mdpi->racks = + rd_tmpabuf_alloc(tbuf, sizeof(char *) * mdpi->racks_cnt); + RD_LIST_FOREACH(rack, curr_list, j) { + mdpi->racks[j] = rack; /* Don't copy, rack points inside + tbuf already*/ + } + rd_list_destroy(curr_list); + } +} + /** * @brief Handle a Metadata response message. * * @param topics are the requested topics (may be NULL) - * @param topic_authorized_operations topics mapped to their acl operations - allowed - * @param cluster_authorized_operations acl operations allowed for cluster - * @param rk_cluster_id returns the current cluster_id (char*) - * @param rk_controller_id returns the current controller_id (int) + * @param request_topics Use when rd_kafka_buf_t* request is NULL * - * The metadata will be marshalled into 'struct rd_kafka_metadata*' structs. + * The metadata will be marshalled into 'rd_kafka_metadata_internal_t *'. + * + * The marshalled metadata is returned in \p *mdip, (NULL on error). * - * The marshalled metadata is returned in \p *mdp, (NULL on error). - * @returns an error code on parse failure, else NO_ERRRO. * * @locality rdkafka main thread */ -rd_kafka_resp_err_t rd_kafka_parse_Metadata( - rd_kafka_broker_t *rkb, - rd_kafka_buf_t *request, - rd_kafka_buf_t *rkbuf, - struct rd_kafka_metadata **mdp, - rd_kafka_topic_authorized_operations_pair_t **topic_authorized_operations, - int32_t *cluster_authorized_operations, - rd_list_t *request_topics, - char **rk_cluster_id, - int *rk_controller_id) { +rd_kafka_resp_err_t +rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *request, + rd_kafka_buf_t *rkbuf, + rd_kafka_metadata_internal_t **mdip, + rd_list_t *request_topics) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; - struct rd_kafka_metadata *md = NULL; + rd_kafka_metadata_internal_t *mdi = NULL; + rd_kafka_metadata_t *md = NULL; size_t rkb_namelen; const int log_decode_errors = LOG_ERR; rd_list_t *missing_topics = NULL; @@ -313,6 +471,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( rd_bool_t cgrp_update = request ? (request->rkbuf_u.Metadata.cgrp_update && rk->rk_cgrp) : rd_false; + rd_bool_t has_reliable_leader_epochs = + rd_kafka_has_reliable_leader_epochs(rkb); const char *reason = request ? (request->rkbuf_u.Metadata.reason ? request->rkbuf_u.Metadata.reason : "(no reason)") @@ -325,12 +485,16 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( 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(); + rd_ts_t ts_start = rd_clock(); + /* If client rack is present, the metadata cache (topic or full) needs + * to contain the partition to rack map. */ + rd_bool_t has_client_rack = rk->rk_conf.client_rack && + RD_KAFKAP_STR_LEN(rk->rk_conf.client_rack); + /* If force_racks is true, the outptr mdip has to contain the partition + * to rack map. */ + rd_bool_t force_rack_computation = + request->rkbuf_u.Metadata.force_racks; + rd_bool_t compute_racks = has_client_rack || force_rack_computation; /* Ignore metadata updates when terminating */ if (rd_kafka_terminating(rkb->rkb_rk)) { @@ -348,17 +512,22 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( rd_kafka_broker_lock(rkb); rkb_namelen = strlen(rkb->rkb_name) + 1; /* We assume that the marshalled representation is - * no more than 4 times larger than the wire representation. */ + * no more than 4 times larger than the wire representation. + * This is increased to 5 times in case if we want to compute partition + * to rack mapping. */ rd_tmpabuf_new(&tbuf, - sizeof(*md) + rkb_namelen + (rkbuf->rkbuf_totlen * 4), + sizeof(*mdi) + rkb_namelen + + (rkbuf->rkbuf_totlen * 4 + + (compute_racks ? rkbuf->rkbuf_totlen : 0)), 0 /*dont assert on fail*/); - if (!(md = rd_tmpabuf_alloc(&tbuf, sizeof(*md)))) { + if (!(mdi = rd_tmpabuf_alloc(&tbuf, sizeof(*mdi)))) { rd_kafka_broker_unlock(rkb); err = RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE; goto err; } + md = &mdi->metadata; md->orig_broker_id = rkb->rkb_nodeid; md->orig_broker_name = rd_tmpabuf_write(&tbuf, rkb->rkb_name, rkb_namelen); @@ -377,15 +546,24 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( "%d brokers: tmpabuf memory shortage", md->broker_cnt); + if (!(mdi->brokers = rd_tmpabuf_alloc( + &tbuf, md->broker_cnt * sizeof(*mdi->brokers)))) + rd_kafka_buf_parse_fail( + rkbuf, "%d internal brokers: tmpabuf memory shortage", + md->broker_cnt); + for (i = 0; i < md->broker_cnt; i++) { rd_kafka_buf_read_i32a(rkbuf, md->brokers[i].id); rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, md->brokers[i].host); rd_kafka_buf_read_i32a(rkbuf, md->brokers[i].port); + mdi->brokers[i].id = md->brokers[i].id; if (ApiVersion >= 1) { - rd_kafkap_str_t rack; - rd_kafka_buf_read_str(rkbuf, &rack); + rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, + mdi->brokers[i].rack_id); + } else { + mdi->brokers[i].rack_id = NULL; } rd_kafka_buf_skip_tags(rkbuf); @@ -393,19 +571,20 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( if (ApiVersion >= 2) { rd_kafka_buf_read_str(rkbuf, &cluster_id); - *rk_cluster_id = RD_KAFKAP_STR_DUP(&cluster_id); + mdi->cluster_id = RD_KAFKAP_STR_DUP(&cluster_id); } if (ApiVersion >= 1) { rd_kafka_buf_read_i32(rkbuf, &controller_id); - *rk_controller_id = controller_id; + mdi->controller_id = controller_id; rd_rkb_dbg(rkb, METADATA, "METADATA", "ClusterId: %.*s, ControllerId: %" PRId32, RD_KAFKAP_STR_PR(&cluster_id), controller_id); } - + qsort(mdi->brokers, md->broker_cnt, sizeof(mdi->brokers[i]), + rd_kafka_metadata_broker_internal_cmp); /* Read TopicMetadata */ rd_kafka_buf_read_arraycnt(rkbuf, &md->topic_cnt, RD_KAFKAP_TOPICS_MAX); @@ -416,13 +595,12 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*md->topics)))) rd_kafka_buf_parse_fail( rkbuf, "%d topics: tmpabuf memory shortage", md->topic_cnt); - - if ((ApiVersion >= 8 && ApiVersion <= 10) && - topic_authorized_operations) { - *topic_authorized_operations = rd_malloc( - sizeof(rd_kafka_topic_authorized_operations_pair_t) * + + if (!(mdi->topics = rd_tmpabuf_alloc(&tbuf, md->topic_cnt * + sizeof(*mdi->topics)))) + rd_kafka_buf_parse_fail( + rkbuf, "%d internal topics: tmpabuf memory shortage", md->topic_cnt); - } for (i = 0; i < md->topic_cnt; i++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].err); @@ -446,16 +624,15 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( 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); - } + if (!(mdi->topics[i].partitions = rd_tmpabuf_alloc( + &tbuf, md->topics[i].partition_cnt * + sizeof(*mdi->topics[i].partitions)))) + rd_kafka_buf_parse_fail(rkbuf, + "%s: %d internal partitions: " + "tmpabuf memory shortage", + md->topics[i].topic, + md->topics[i].partition_cnt); + for (j = 0; j < md->topics[i].partition_cnt; j++) { rd_kafka_buf_read_i16a(rkbuf, @@ -464,12 +641,22 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( md->topics[i].partitions[j].id); rd_kafka_buf_read_i32a( rkbuf, md->topics[i].partitions[j].leader); + + mdi->topics[i].partitions[j].id = + md->topics[i].partitions[j].id; 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); + rkbuf, + &mdi->topics[i].partitions[j].leader_epoch); + if (!has_reliable_leader_epochs) + mdi->topics[i] + .partitions[j] + .leader_epoch = -1; + } else { + mdi->topics[i].partitions[j].leader_epoch = -1; } + mdi->topics[i].partitions[j].racks_cnt = 0; + mdi->topics[i].partitions[j].racks = NULL; /* Replicas */ rd_kafka_buf_read_arraycnt( @@ -546,10 +733,9 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( /* TopicAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &TopicAuthorizedOperations); - (*topic_authorized_operations)[i].topic_name = - md->topics[i].topic; - (*topic_authorized_operations)[i] - .authorized_operations = TopicAuthorizedOperations; + mdi->topics[i].topic_name = md->topics[i].topic; + mdi->topics[i].topic_authorized_operations = + TopicAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -566,37 +752,20 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( continue; } - 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 - * 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, 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); - } + qsort(mdi->topics[i].partitions, md->topics[i].partition_cnt, + sizeof(*mdi->topics[i].partitions), + rd_kafka_metadata_partition_internal_cmp); + + if (compute_racks) + rd_kafka_populate_metadata_topic_racks(&tbuf, i, mdi); /* Update topic state based on the topic metadata */ rd_kafka_parse_Metadata_update_topic(rkb, &md->topics[i], - leader_epochs); + &mdi->topics[i]); if (requested_topics) { @@ -610,8 +779,15 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( rd_kafka_wrlock(rk); rd_kafka_metadata_cache_topic_update( - rk, &md->topics[i], - rd_false /*propagate later*/); + rk, &md->topics[i], &mdi->topics[i], + rd_false /*propagate later*/, + /* use has_client_rack rather than + compute_racks. We need cached rack ids + only in case we need to rejoin the group + if they change and client.rack is set + (KIP-881). */ + has_client_rack, mdi->brokers, + md->broker_cnt); cache_changes++; rd_kafka_wrunlock(rk); } @@ -622,7 +798,8 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( int32_t ClusterAuthorizedOperations; /* ClusterAuthorizedOperations */ rd_kafka_buf_read_i32(rkbuf, &ClusterAuthorizedOperations); - *cluster_authorized_operations = ClusterAuthorizedOperations; + mdi->cluster_authorized_operations = + ClusterAuthorizedOperations; } rd_kafka_buf_skip_tags(rkbuf); @@ -725,9 +902,18 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( if (rkb->rkb_rk->rk_full_metadata) rd_kafka_metadata_destroy( - rkb->rkb_rk->rk_full_metadata); - rkb->rkb_rk->rk_full_metadata = - rd_kafka_metadata_copy(md, tbuf.of); + &rkb->rkb_rk->rk_full_metadata->metadata); + + /* use has_client_rack rather than compute_racks. We need cached + * rack ids only in case we need to rejoin the group if they + * change and client.rack is set (KIP-881). */ + if (has_client_rack) + rkb->rkb_rk->rk_full_metadata = + rd_kafka_metadata_copy_add_racks(mdi, tbuf.of); + else + rkb->rkb_rk->rk_full_metadata = + rd_kafka_metadata_copy(mdi, tbuf.of); + rkb->rkb_rk->rk_ts_full_metadata = rkb->rkb_rk->rk_ts_metadata; rd_rkb_dbg(rkb, METADATA, "METADATA", "Caching full metadata with " @@ -773,16 +959,13 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( 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, * the requestee will do. * The tbuf is explicitly not destroyed as we return its memory * to the caller. */ - *mdp = md; + *mdip = mdi; return RD_KAFKA_RESP_ERR_NO_ERROR; @@ -799,10 +982,6 @@ rd_kafka_resp_err_t rd_kafka_parse_Metadata( if (missing_topics) rd_list_destroy(missing_topics); - - if (leader_epochs) - rd_free(leader_epochs); - rd_tmpabuf_destroy(&tbuf); return err; @@ -830,12 +1009,15 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk, rd_kafka_topic_partition_list_t *errored) { int ti, i; size_t cnt = 0; - const struct rd_kafka_metadata *metadata; + rd_kafka_metadata_internal_t *mdi; + struct rd_kafka_metadata *metadata; rd_kafka_topic_partition_list_t *unmatched; rd_kafka_rdlock(rk); - metadata = rk->rk_full_metadata; - if (!metadata) { + mdi = rk->rk_full_metadata; + metadata = &mdi->metadata; + + if (!mdi) { rd_kafka_rdunlock(rk); return 0; } @@ -876,10 +1058,11 @@ rd_kafka_metadata_topic_match(rd_kafka_t *rk, continue; /* Skip errored topics */ } - rd_list_add( - tinfos, - rd_kafka_topic_info_new( - topic, metadata->topics[ti].partition_cnt)); + rd_list_add(tinfos, + rd_kafka_topic_info_new_with_rack( + topic, + metadata->topics[ti].partition_cnt, + mdi->topics[ti].partitions)); cnt++; } @@ -924,16 +1107,18 @@ rd_kafka_metadata_topic_filter(rd_kafka_t *rk, rd_kafka_rdlock(rk); /* For each topic in match, look up the topic in the cache. */ for (i = 0; i < match->cnt; i++) { - const char *topic = match->elems[i].topic; - const rd_kafka_metadata_topic_t *mtopic; + const char *topic = match->elems[i].topic; + const rd_kafka_metadata_topic_t *mtopic = NULL; /* Ignore topics in blacklist */ if (rk->rk_conf.topic_blacklist && rd_kafka_pattern_match(rk->rk_conf.topic_blacklist, topic)) continue; - mtopic = - rd_kafka_metadata_cache_topic_get(rk, topic, 1 /*valid*/); + struct rd_kafka_metadata_cache_entry *rkmce = + rd_kafka_metadata_cache_find(rk, topic, 1 /* valid */); + if (rkmce) + mtopic = &rkmce->rkmce_mtopic; if (!mtopic) rd_kafka_topic_partition_list_add(errored, topic, @@ -944,8 +1129,11 @@ rd_kafka_metadata_topic_filter(rd_kafka_t *rk, RD_KAFKA_PARTITION_UA) ->err = mtopic->err; else { - rd_list_add(tinfos, rd_kafka_topic_info_new( - topic, mtopic->partition_cnt)); + rd_list_add(tinfos, + rd_kafka_topic_info_new_with_rack( + topic, mtopic->partition_cnt, + rkmce->rkmce_metadata_internal_topic + .partitions)); cnt++; } @@ -1075,11 +1263,11 @@ rd_kafka_metadata_refresh_topics(rd_kafka_t *rk, "Requesting metadata for %d/%d topics: %s", rd_list_cnt(&q_topics), rd_list_cnt(topics), reason); - rd_kafka_MetadataRequest( - rkb, &q_topics, reason, allow_auto_create, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, cgrp_update, NULL, - NULL, 0, NULL); + rd_kafka_MetadataRequest(rkb, &q_topics, reason, allow_auto_create, + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + cgrp_update, rd_false /* force_racks */, NULL, + NULL, 0, NULL); rd_list_destroy(&q_topics); @@ -1258,9 +1446,10 @@ rd_kafka_resp_err_t rd_kafka_metadata_refresh_all(rd_kafka_t *rk, rd_list_init(&topics, 0, NULL); /* empty list = all topics */ rd_kafka_MetadataRequest( rkb, &topics, reason, rd_false /*no auto create*/, - rd_true /*include cluster authorized operations */, - rd_true /*include topic authorized operations */, - rd_true /*cgrp update*/, NULL, NULL, 0, NULL); + rd_false /*!include cluster authorized operations */, + rd_false /*!include topic authorized operations */, + rd_true /*cgrp update*/, rd_false /* force_rack */, + NULL, NULL, 0, NULL); rd_list_destroy(&topics); if (destroy_rkb) @@ -1302,7 +1491,7 @@ rd_kafka_metadata_request(rd_kafka_t *rk, rd_kafka_MetadataRequest(rkb, topics, reason, allow_auto_create_topics, include_cluster_authorized_operations, include_topic_authorized_operations, - cgrp_update, rko, NULL, 0, NULL); + cgrp_update, rd_false /* force racks */, rko, NULL, 0, NULL); if (destroy_rkb) rd_kafka_broker_destroy(rkb); @@ -1417,20 +1606,30 @@ void rd_kafka_metadata_fast_leader_query(rd_kafka_t *rk) { * * @param topics elements are checked for .topic and .partition_cnt * @param topic_cnt is the number of topic elements in \p topics. + * @param replication_factor is the number of replicas of each partition (set to + * -1 to ignore). + * @param num_brokers is the number of brokers in the cluster. * * @returns a newly allocated metadata object that must be freed with * rd_kafka_metadata_destroy(). * + * @note \p replication_factor and \p num_brokers must be used together for + * setting replicas of each partition. + * * @sa rd_kafka_metadata_copy() */ rd_kafka_metadata_t * rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, - size_t topic_cnt) { + size_t topic_cnt, + int replication_factor, + int num_brokers) { + rd_kafka_metadata_internal_t *mdi; rd_kafka_metadata_t *md; rd_tmpabuf_t tbuf; size_t topic_names_size = 0; int total_partition_cnt = 0; size_t i; + int curr_broker = 0; /* Calculate total partition count and topic names size before * allocating memory. */ @@ -1439,22 +1638,37 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, total_partition_cnt += topics[i].partition_cnt; } + /* If the replication factor is given, num_brokers must also be given */ + rd_assert(replication_factor <= 0 || num_brokers > 0); /* Allocate contiguous buffer which will back all the memory * needed by the final metadata_t object */ rd_tmpabuf_new( &tbuf, - sizeof(*md) + (sizeof(*md->topics) * topic_cnt) + topic_names_size + - (64 /*topic name size..*/ * topic_cnt) + - (sizeof(*md->topics[0].partitions) * total_partition_cnt), + sizeof(*mdi) + (sizeof(*md->topics) * topic_cnt) + + topic_names_size + (64 /*topic name size..*/ * topic_cnt) + + (sizeof(*md->topics[0].partitions) * total_partition_cnt) + + (sizeof(*mdi->topics) * topic_cnt) + + (sizeof(*mdi->topics[0].partitions) * total_partition_cnt) + + (sizeof(*mdi->brokers) * RD_ROUNDUP(num_brokers, 8)) + + (replication_factor > 0 ? RD_ROUNDUP(replication_factor, 8) * + total_partition_cnt * sizeof(int) + : 0), 1 /*assert on fail*/); - md = rd_tmpabuf_alloc(&tbuf, sizeof(*md)); - memset(md, 0, sizeof(*md)); + mdi = rd_tmpabuf_alloc(&tbuf, sizeof(*mdi)); + memset(mdi, 0, sizeof(*mdi)); + md = &mdi->metadata; md->topic_cnt = (int)topic_cnt; md->topics = rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*md->topics)); + mdi->topics = + rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*mdi->topics)); + + md->broker_cnt = num_brokers; + mdi->brokers = + rd_tmpabuf_alloc(&tbuf, md->broker_cnt * sizeof(*mdi->brokers)); for (i = 0; i < (size_t)md->topic_cnt; i++) { int j; @@ -1467,12 +1681,42 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, md->topics[i].partitions = rd_tmpabuf_alloc( &tbuf, md->topics[i].partition_cnt * sizeof(*md->topics[i].partitions)); + mdi->topics[i].partitions = rd_tmpabuf_alloc( + &tbuf, md->topics[i].partition_cnt * + sizeof(*mdi->topics[i].partitions)); for (j = 0; j < md->topics[i].partition_cnt; j++) { + int k; memset(&md->topics[i].partitions[j], 0, sizeof(md->topics[i].partitions[j])); - md->topics[i].partitions[j].id = j; + memset(&mdi->topics[i].partitions[j], 0, + sizeof(mdi->topics[i].partitions[j])); + md->topics[i].partitions[j].id = j; + mdi->topics[i].partitions[j].id = j; + mdi->topics[i].partitions[j].leader_epoch = -1; + mdi->topics[i].partitions[j].racks_cnt = 0; + mdi->topics[i].partitions[j].racks = NULL; + md->topics[i].partitions[j].id = j; + + /* In case replication_factor is not given, don't set + * replicas. */ + if (replication_factor <= 0) + continue; + + md->topics[i].partitions[j].replicas = rd_tmpabuf_alloc( + &tbuf, replication_factor * sizeof(int)); + md->topics[i].partitions[j].leader = curr_broker; + md->topics[i].partitions[j].replica_cnt = + replication_factor; + for (k = 0; k < replication_factor; k++) { + md->topics[i].partitions[j].replicas[k] = + (j + k + curr_broker) % num_brokers; + } } + if (num_brokers > 0) + curr_broker = + (curr_broker + md->topics[i].partition_cnt) % + num_brokers; } /* Check for tmpabuf errors */ @@ -1484,6 +1728,24 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, return md; } +/* Implementation for rd_kafka_metadata_new_topic*mockv() */ +static rd_kafka_metadata_t * +rd_kafka_metadata_new_topic_mockv_internal(size_t topic_cnt, + int replication_factor, + int num_brokers, + va_list args) { + rd_kafka_metadata_topic_t *topics; + size_t i; + + topics = rd_alloca(sizeof(*topics) * topic_cnt); + for (i = 0; i < topic_cnt; i++) { + topics[i].topic = va_arg(args, char *); + topics[i].partition_cnt = va_arg(args, int); + } + + return rd_kafka_metadata_new_topic_mock( + topics, topic_cnt, replication_factor, num_brokers); +} /** * @brief Create mock Metadata (for testing) based on the @@ -1497,18 +1759,75 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, * @sa rd_kafka_metadata_new_topic_mock() */ rd_kafka_metadata_t *rd_kafka_metadata_new_topic_mockv(size_t topic_cnt, ...) { - rd_kafka_metadata_topic_t *topics; + rd_kafka_metadata_t *metadata; va_list ap; - size_t i; - topics = rd_alloca(sizeof(*topics) * topic_cnt); + va_start(ap, topic_cnt); + metadata = + rd_kafka_metadata_new_topic_mockv_internal(topic_cnt, -1, 0, ap); + va_end(ap); + + return metadata; +} + +/** + * @brief Create mock Metadata (for testing) based on the + * var-arg tuples of (const char *topic, int partition_cnt). + * + * @param replication_factor is the number of replicas of each partition. + * @param num_brokers is the number of brokers in the cluster. + * @param topic_cnt is the number of topic,partition_cnt tuples. + * + * @returns a newly allocated metadata object that must be freed with + * rd_kafka_metadata_destroy(). + * + * @sa rd_kafka_metadata_new_topic_mock() + */ +rd_kafka_metadata_t *rd_kafka_metadata_new_topic_with_partition_replicas_mockv( + int replication_factor, + int num_brokers, + size_t topic_cnt, + ...) { + rd_kafka_metadata_t *metadata; + va_list ap; va_start(ap, topic_cnt); + metadata = rd_kafka_metadata_new_topic_mockv_internal( + topic_cnt, replication_factor, num_brokers, ap); + va_end(ap); + + return metadata; +} + +/** + * @brief Create mock Metadata (for testing) based on arrays topic_names and + * partition_cnts. + * + * @param replication_factor is the number of replicas of each partition. + * @param num_brokers is the number of brokers in the cluster. + * @param topic_names names of topics. + * @param partition_cnts number of partitions in each topic. + * @param topic_cnt number of topics. + * + * @return rd_kafka_metadata_t* + * + * @sa rd_kafka_metadata_new_topic_mock() + */ +rd_kafka_metadata_t * +rd_kafka_metadata_new_topic_with_partition_replicas_mock(int replication_factor, + int num_brokers, + char *topic_names[], + int *partition_cnts, + size_t topic_cnt) { + rd_kafka_metadata_topic_t *topics; + size_t i; + + topics = rd_alloca(sizeof(*topics) * topic_cnt); for (i = 0; i < topic_cnt; i++) { - topics[i].topic = va_arg(ap, char *); - topics[i].partition_cnt = va_arg(ap, int); + topics[i].topic = topic_names[i]; + topics[i].partition_cnt = partition_cnts[i]; } - va_end(ap); - return rd_kafka_metadata_new_topic_mock(topics, topic_cnt); + return rd_kafka_metadata_new_topic_mock( + topics, topic_cnt, replication_factor, num_brokers); } diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h index 9344e470b1..464fab5b8d 100644 --- a/src/rdkafka_metadata.h +++ b/src/rdkafka_metadata.h @@ -31,21 +31,85 @@ #include "rdavl.h" +/** + * @brief Metadata partition internal container + */ +typedef struct rd_kafka_metadata_partition_internal_s { + /** Partition Id */ + int32_t id; + /** Partition leader epoch */ + int32_t leader_epoch; + /* Racks for this partition. Sorted and de-duplicated. */ + char **racks; + /* Count of the racks */ + size_t racks_cnt; +} rd_kafka_metadata_partition_internal_t; + +/** + * @brief Metadata topic internal container + */ +typedef struct rd_kafka_metadata_topic_internal_s { + /** Internal metadata partition structs. + * same count as metadata.topics[i].partition_cnt. + * Sorted by Partition Id. */ + rd_kafka_metadata_partition_internal_t *partitions; + /** Topic Name. */ + char* topic_name; + int32_t *topic_authorized_operations; /**< ACL operations allowed + for topic */ +} rd_kafka_metadata_topic_internal_t; + + +/** + * @brief Metadata broker internal container + */ +typedef struct rd_kafka_metadata_broker_internal_s { + /** Broker Id. */ + int32_t id; + /** Rack Id (optional). */ + char *rack_id; +} rd_kafka_metadata_broker_internal_t; + +/** + * @brief Metadata internal container + */ +typedef struct rd_kafka_metadata_internal_s { + rd_kafka_metadata_t + metadata; /**< Public metadata struct. Must + be kept the first field so the pointer + can be cast to *rd_kafka_metadata_internal_t + when needed */ + /* Internal metadata brokers. Same count as metadata.broker_cnt. + * Sorted by broker id. */ + rd_kafka_metadata_broker_internal_t *brokers; + /* Internal metadata topics. Same count as metadata.topic_cnt. */ + rd_kafka_metadata_topic_internal_t *topics; + char *cluster_id; /**< current cluster id in \p cluster*/ + int controller_id; /**< current controller id in \p cluster*/ + rd_list_t *cluster_authorized_operations; /**< ACL operations allowed + for cluster */ +} rd_kafka_metadata_internal_t; + +/** + * @brief The internal metadata type corresponding to the + * public one. + */ +#define rd_kafka_metadata_get_internal(md) ((rd_kafka_metadata_internal_t *)md) + 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, - struct rd_kafka_metadata **mdp, - rd_kafka_topic_authorized_operations_pair_t **topic_authorized_operations, - int32_t *cluster_authorized_operations, - rd_list_t *request_topics, - char **rk_cluster_id, - int *rk_controller_id); +rd_kafka_resp_err_t rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *request, + rd_kafka_buf_t *rkbuf, + rd_kafka_metadata_internal_t **mdp, + rd_list_t *request_topics); -struct rd_kafka_metadata * -rd_kafka_metadata_copy(const struct rd_kafka_metadata *md, size_t size); +rd_kafka_metadata_internal_t * +rd_kafka_metadata_copy(const rd_kafka_metadata_internal_t *mdi, size_t size); + +rd_kafka_metadata_internal_t * +rd_kafka_metadata_copy_add_racks(const rd_kafka_metadata_internal_t *mdi, + size_t size); size_t rd_kafka_metadata_topic_match(rd_kafka_t *rk, @@ -103,21 +167,37 @@ rd_kafka_metadata_request(rd_kafka_t *rk, int rd_kafka_metadata_partition_id_cmp(const void *_a, const void *_b); +int rd_kafka_metadata_broker_internal_cmp(const void *_a, const void *_b); + + +#define rd_kafka_metadata_broker_internal_find(mdi, broker_id, broker) \ + do { \ + rd_kafka_metadata_broker_internal_t __key = {.id = broker_id}; \ + broker = \ + bsearch(&__key, mdi->brokers, mdi->metadata.broker_cnt, \ + sizeof(rd_kafka_metadata_broker_internal_t), \ + rd_kafka_metadata_broker_internal_cmp); \ + } while (0) + + rd_kafka_metadata_t * rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics, - size_t topic_cnt); + size_t topic_cnt, + int replication_factor, + int num_brokers); rd_kafka_metadata_t *rd_kafka_metadata_new_topic_mockv(size_t topic_cnt, ...); +rd_kafka_metadata_t *rd_kafka_metadata_new_topic_with_partition_replicas_mockv( + int replication_factor, + int num_brokers, + size_t topic_cnt, + ...); +rd_kafka_metadata_t * +rd_kafka_metadata_new_topic_with_partition_replicas_mock(int replication_factor, + int num_brokers, + char *topic_names[], + int *partition_cnts, + size_t topic_cnt); -/** - * @{ - * - * @brief Metadata topic name and authorized operations pair - */ - -typedef struct rd_kafka_topic_authorized_operations_pair { - char *topic_name; - int32_t authorized_operations; -} rd_kafka_topic_authorized_operations_pair; /** * @{ * @@ -132,6 +212,8 @@ struct rd_kafka_metadata_cache_entry { /** 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 */ + /* Cached internal topic metadata */ + rd_kafka_metadata_topic_internal_t rkmce_metadata_internal_topic; /* rkmce_topics.partitions memory points here. */ }; @@ -174,12 +256,14 @@ 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); -void rd_kafka_metadata_cache_update(rd_kafka_t *rk, - const rd_kafka_metadata_t *md, - int abs_update); +void rd_kafka_metadata_cache_topic_update( + rd_kafka_t *rk, + const rd_kafka_metadata_topic_t *mdt, + const rd_kafka_metadata_topic_internal_t *mdit, + rd_bool_t propagate, + rd_bool_t include_metadata, + rd_kafka_metadata_broker_internal_t *brokers, + size_t broker_cnt); void rd_kafka_metadata_cache_propagate_changes(rd_kafka_t *rk); struct rd_kafka_metadata_cache_entry * rd_kafka_metadata_cache_find(rd_kafka_t *rk, const char *topic, int valid); diff --git a/src/rdkafka_metadata_cache.c b/src/rdkafka_metadata_cache.c index 514d391a83..d579301b79 100644 --- a/src/rdkafka_metadata_cache.c +++ b/src/rdkafka_metadata_cache.c @@ -238,13 +238,18 @@ int rd_kafka_metadata_partition_id_cmp(const void *_a, const void *_b) { * * @locks_required rd_kafka_wrlock() */ -static struct rd_kafka_metadata_cache_entry * -rd_kafka_metadata_cache_insert(rd_kafka_t *rk, - const rd_kafka_metadata_topic_t *mtopic, - rd_ts_t now, - rd_ts_t ts_expires) { +static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert( + rd_kafka_t *rk, + const rd_kafka_metadata_topic_t *mtopic, + const rd_kafka_metadata_topic_internal_t *metadata_internal_topic, + rd_ts_t now, + rd_ts_t ts_expires, + rd_bool_t include_racks, + rd_kafka_metadata_broker_internal_t *brokers_internal, + size_t broker_cnt) { struct rd_kafka_metadata_cache_entry *rkmce, *old; size_t topic_len; + size_t racks_size = 0; rd_tmpabuf_t tbuf; int i; @@ -253,19 +258,43 @@ rd_kafka_metadata_cache_insert(rd_kafka_t *rk, * rd_tmpabuf_t provides the infrastructure to do this. * Because of this we copy all the structs verbatim but * any pointer fields needs to be copied explicitly to update - * the pointer address. */ + * the pointer address. + * See also rd_kafka_metadata_cache_delete which frees this. */ topic_len = strlen(mtopic->topic) + 1; - rd_tmpabuf_new(&tbuf, - RD_ROUNDUP(sizeof(*rkmce), 8) + - RD_ROUNDUP(topic_len, 8) + - (mtopic->partition_cnt * - RD_ROUNDUP(sizeof(*mtopic->partitions), 8)), - 1 /*assert on fail*/); + + for (i = 0; include_racks && i < mtopic->partition_cnt; i++) { + size_t j; + racks_size += RD_ROUNDUP( + metadata_internal_topic->partitions[i].racks_cnt * + sizeof(char *), + 8); + for (j = 0; + j < metadata_internal_topic->partitions[i].racks_cnt; + j++) { + racks_size += RD_ROUNDUP( + strlen(metadata_internal_topic->partitions[i] + .racks[j]) + + 1, + 8); + } + } + + rd_tmpabuf_new( + &tbuf, + RD_ROUNDUP(sizeof(*rkmce), 8) + RD_ROUNDUP(topic_len, 8) + + (mtopic->partition_cnt * + RD_ROUNDUP(sizeof(*mtopic->partitions), 8)) + + (mtopic->partition_cnt * + RD_ROUNDUP(sizeof(*metadata_internal_topic->partitions), 8)) + + racks_size, + 1 /*assert on fail*/); rkmce = rd_tmpabuf_alloc(&tbuf, sizeof(*rkmce)); rkmce->rkmce_mtopic = *mtopic; + rkmce->rkmce_metadata_internal_topic = *metadata_internal_topic; + /* Copy topic name and update pointer */ rkmce->rkmce_mtopic.topic = rd_tmpabuf_write_str(&tbuf, mtopic->topic); @@ -274,6 +303,41 @@ rd_kafka_metadata_cache_insert(rd_kafka_t *rk, &tbuf, mtopic->partitions, mtopic->partition_cnt * sizeof(*mtopic->partitions)); + /* Copy partition array (internal) and update pointer */ + rkmce->rkmce_metadata_internal_topic.partitions = + rd_tmpabuf_write(&tbuf, metadata_internal_topic->partitions, + mtopic->partition_cnt * + sizeof(*metadata_internal_topic->partitions)); + + + /* Sort partitions for future bsearch() lookups. */ + qsort(rkmce->rkmce_mtopic.partitions, rkmce->rkmce_mtopic.partition_cnt, + sizeof(*rkmce->rkmce_mtopic.partitions), + rd_kafka_metadata_partition_id_cmp); + + /* partitions (internal) are already sorted. */ + + if (include_racks) { + for (i = 0; i < rkmce->rkmce_mtopic.partition_cnt; i++) { + size_t j; + rd_kafka_metadata_partition_t *mdp = + &rkmce->rkmce_mtopic.partitions[i]; + rd_kafka_metadata_partition_internal_t *mdpi = + &rkmce->rkmce_metadata_internal_topic.partitions[i]; + rd_kafka_metadata_partition_internal_t *mdpi_orig = + &metadata_internal_topic->partitions[i]; + + if (mdp->replica_cnt == 0 || mdpi->racks_cnt == 0) + continue; + + mdpi->racks = rd_tmpabuf_alloc( + &tbuf, sizeof(char *) * mdpi->racks_cnt); + for (j = 0; j < mdpi_orig->racks_cnt; j++) + mdpi->racks[j] = rd_tmpabuf_write_str( + &tbuf, mdpi_orig->racks[j]); + } + } + /* Clear uncached fields. */ for (i = 0; i < mtopic->partition_cnt; i++) { rkmce->rkmce_mtopic.partitions[i].replicas = NULL; @@ -281,12 +345,6 @@ rd_kafka_metadata_cache_insert(rd_kafka_t *rk, rkmce->rkmce_mtopic.partitions[i].isrs = NULL; rkmce->rkmce_mtopic.partitions[i].isr_cnt = 0; } - - /* Sort partitions for future bsearch() lookups. */ - qsort(rkmce->rkmce_mtopic.partitions, rkmce->rkmce_mtopic.partition_cnt, - sizeof(*rkmce->rkmce_mtopic.partitions), - rd_kafka_metadata_partition_id_cmp); - TAILQ_INSERT_TAIL(&rk->rk_metadata_cache.rkmc_expiry, rkmce, rkmce_link); rk->rk_metadata_cache.rkmc_cnt++; @@ -365,9 +423,14 @@ void rd_kafka_metadata_cache_expiry_start(rd_kafka_t *rk) { * * @locks rd_kafka_wrlock() */ -void rd_kafka_metadata_cache_topic_update(rd_kafka_t *rk, - const rd_kafka_metadata_topic_t *mdt, - rd_bool_t propagate) { +void rd_kafka_metadata_cache_topic_update( + rd_kafka_t *rk, + const rd_kafka_metadata_topic_t *mdt, + const rd_kafka_metadata_topic_internal_t *mdit, + rd_bool_t propagate, + rd_bool_t include_racks, + rd_kafka_metadata_broker_internal_t *brokers, + size_t broker_cnt) { rd_ts_t now = rd_clock(); rd_ts_t ts_expires = now + (rk->rk_conf.metadata_max_age_ms * 1000); int changed = 1; @@ -380,7 +443,9 @@ void rd_kafka_metadata_cache_topic_update(rd_kafka_t *rk, if (!mdt->err || mdt->err == RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED || mdt->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART) - rd_kafka_metadata_cache_insert(rk, mdt, now, ts_expires); + rd_kafka_metadata_cache_insert(rk, mdt, mdit, now, ts_expires, + include_racks, brokers, + broker_cnt); else changed = rd_kafka_metadata_cache_delete_by_name(rk, mdt->topic); @@ -390,45 +455,6 @@ void rd_kafka_metadata_cache_topic_update(rd_kafka_t *rk, } -/** - * @brief Update the metadata cache with the provided metadata. - * - * @param abs_update int: absolute update: purge cache before updating. - * - * @locks rd_kafka_wrlock() - */ -void rd_kafka_metadata_cache_update(rd_kafka_t *rk, - const rd_kafka_metadata_t *md, - int abs_update) { - struct rd_kafka_metadata_cache_entry *rkmce; - rd_ts_t now = rd_clock(); - rd_ts_t ts_expires = now + (rk->rk_conf.metadata_max_age_ms * 1000); - int i; - - rd_kafka_dbg(rk, METADATA, "METADATA", - "%s of metadata cache with %d topic(s)", - abs_update ? "Absolute update" : "Update", md->topic_cnt); - - if (abs_update) - rd_kafka_metadata_cache_purge(rk, rd_false /*not observers*/); - - - for (i = 0; i < md->topic_cnt; i++) - rd_kafka_metadata_cache_insert(rk, &md->topics[i], now, - ts_expires); - - /* Update expiry timer */ - if ((rkmce = TAILQ_FIRST(&rk->rk_metadata_cache.rkmc_expiry))) - rd_kafka_timer_start(&rk->rk_timers, - &rk->rk_metadata_cache.rkmc_expiry_tmr, - rkmce->rkmce_ts_expires - now, - rd_kafka_metadata_cache_evict_tmr_cb, rk); - - if (md->topic_cnt > 0 || abs_update) - rd_kafka_metadata_cache_propagate_changes(rk); -} - - /** * @brief Remove cache hints for topics in \p topics * This is done when the Metadata response has been parsed and @@ -499,6 +525,8 @@ int rd_kafka_metadata_cache_hint(rd_kafka_t *rk, RD_LIST_FOREACH(topic, topics, i) { rd_kafka_metadata_topic_t mtopic = {.topic = (char *)topic, .err = err}; + rd_kafka_metadata_topic_internal_t metadata_internal_topic = + RD_ZERO_INIT; /*const*/ struct rd_kafka_metadata_cache_entry *rkmce; /* !replace: Dont overwrite valid entries */ @@ -512,7 +540,9 @@ int rd_kafka_metadata_cache_hint(rd_kafka_t *rk, /* FALLTHRU */ } - rd_kafka_metadata_cache_insert(rk, &mtopic, now, ts_expires); + rd_kafka_metadata_cache_insert(rk, &mtopic, + &metadata_internal_topic, now, + ts_expires, rd_false, NULL, 0); cnt++; if (dst) diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index ae7940533c..d4bf595a3a 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -424,12 +424,15 @@ int64_t rd_kafka_mock_partition_offset_for_leader_epoch( * @brief Automatically assign replicas for partition */ static void -rd_kafka_mock_partition_assign_replicas(rd_kafka_mock_partition_t *mpart) { +rd_kafka_mock_partition_assign_replicas(rd_kafka_mock_partition_t *mpart, + int replication_factor) { rd_kafka_mock_cluster_t *mcluster = mpart->topic->cluster; - int replica_cnt = - RD_MIN(mcluster->defaults.replication_factor, mcluster->broker_cnt); + int replica_cnt = RD_MIN(replication_factor, mcluster->broker_cnt); rd_kafka_mock_broker_t *mrkb; int i = 0; + int first_replica = + (mpart->id * replication_factor) % mcluster->broker_cnt; + int skipped = 0; if (mpart->replicas) rd_free(mpart->replicas); @@ -437,7 +440,19 @@ rd_kafka_mock_partition_assign_replicas(rd_kafka_mock_partition_t *mpart) { mpart->replicas = rd_calloc(replica_cnt, sizeof(*mpart->replicas)); mpart->replica_cnt = replica_cnt; - /* FIXME: randomize this using perhaps reservoir sampling */ + + /* Use a predictable, determininistic order on a per-topic basis. + * + * Two loops are needed for wraparound. */ + TAILQ_FOREACH(mrkb, &mcluster->brokers, link) { + if (skipped < first_replica) { + skipped++; + continue; + } + if (i == mpart->replica_cnt) + break; + mpart->replicas[i++] = mrkb; + } TAILQ_FOREACH(mrkb, &mcluster->brokers, link) { if (i == mpart->replica_cnt) break; @@ -562,7 +577,7 @@ static void rd_kafka_mock_partition_init(rd_kafka_mock_topic_t *mtopic, rd_list_init(&mpart->pidstates, 0, rd_free); - rd_kafka_mock_partition_assign_replicas(mpart); + rd_kafka_mock_partition_assign_replicas(mpart, replication_factor); } rd_kafka_mock_partition_t * @@ -1470,6 +1485,9 @@ static void rd_kafka_mock_broker_destroy(rd_kafka_mock_broker_t *mrkb) { rd_kafka_mock_error_stack_destroy(errstack); } + if (mrkb->rack) + rd_free(mrkb->rack); + TAILQ_REMOVE(&mrkb->cluster->brokers, mrkb, link); mrkb->cluster->broker_cnt--; diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index 8f71fb48c9..3fa5367793 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -294,15 +294,30 @@ static void rd_kafka_mock_cgrp_elect_leader(rd_kafka_mock_cgrp_t *mcgrp) { mcgrp->generation_id++; - /* Elect a leader. - * FIXME: For now we'll use the first member */ - mcgrp->leader = TAILQ_FIRST(&mcgrp->members); + /* Elect a leader deterministically if the group.instance.id is + * available, using the lexicographic order of group.instance.ids. + * This is not how it's done on a real broker, which uses the first + * member joined. But we use a determinstic method for better testing, + * (in case we want to enforce a some consumer to be the group leader). + * If group.instance.id is not specified for any consumer, we use the + * first one joined, similar to the real broker. */ + mcgrp->leader = NULL; + TAILQ_FOREACH(member, &mcgrp->members, link) { + if (!mcgrp->leader) + mcgrp->leader = member; + else if (mcgrp->leader->group_instance_id && + member->group_instance_id && + (rd_strcmp(mcgrp->leader->group_instance_id, + member->group_instance_id) > 0)) + mcgrp->leader = member; + } - rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK", - "Consumer group %s with %d member(s) is rebalancing: " - "elected leader is %s, generation id %d", - mcgrp->id, mcgrp->member_cnt, mcgrp->leader->id, - mcgrp->generation_id); + rd_kafka_dbg( + mcgrp->cluster->rk, MOCK, "MOCK", + "Consumer group %s with %d member(s) is rebalancing: " + "elected leader is %s (group.instance.id = %s), generation id %d", + mcgrp->id, mcgrp->member_cnt, mcgrp->leader->id, + mcgrp->leader->group_instance_id, mcgrp->generation_id); /* Find the most commonly supported protocol name among the members. * FIXME: For now we'll blindly use the first protocol of the leader. */ @@ -525,6 +540,7 @@ rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp, rd_kafka_buf_t *resp, const rd_kafkap_str_t *MemberId, const rd_kafkap_str_t *ProtocolType, + const rd_kafkap_str_t *GroupInstanceId, rd_kafka_mock_cgrp_proto_t *protos, int proto_cnt, int session_timeout_ms) { @@ -549,6 +565,10 @@ rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp, } else member->id = RD_KAFKAP_STR_DUP(MemberId); + if (GroupInstanceId) + member->group_instance_id = + RD_KAFKAP_STR_DUP(GroupInstanceId); + TAILQ_INSERT_TAIL(&mcgrp->members, member, link); mcgrp->member_cnt++; } diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 3a004d41db..59d40e421a 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -1199,8 +1199,8 @@ static int rd_kafka_mock_handle_JoinGroup(rd_kafka_mock_connection_t *mconn, /* This triggers an async rebalance, the response will be * sent later. */ err = rd_kafka_mock_cgrp_member_add( - mcgrp, mconn, resp, &MemberId, &ProtocolType, protos, - ProtocolCnt, SessionTimeoutMs); + mcgrp, mconn, resp, &MemberId, &ProtocolType, + &GroupInstanceId, protos, ProtocolCnt, SessionTimeoutMs); if (!err) { /* .._add() assumes ownership of resp and protos */ protos = NULL; diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index ea3b6cab4d..1f33476aff 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -505,6 +505,7 @@ rd_kafka_mock_cgrp_member_add(rd_kafka_mock_cgrp_t *mcgrp, rd_kafka_mock_connection_t *mconn, rd_kafka_buf_t *resp, const rd_kafkap_str_t *MemberId, + const rd_kafkap_str_t *GroupInstanceId, const rd_kafkap_str_t *ProtocolType, rd_kafka_mock_cgrp_proto_t *protos, int proto_cnt, diff --git a/src/rdkafka_msg.c b/src/rdkafka_msg.c index 17b67999bb..a433e29b15 100644 --- a/src/rdkafka_msg.c +++ b/src/rdkafka_msg.c @@ -1562,8 +1562,8 @@ 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 || + if (unlikely(!rkmessage->rkt || rd_kafka_rkt_is_lw(rkmessage->rkt) || + !rkmessage->rkt->rkt_rk || rkmessage->rkt->rkt_rk->rk_type != RD_KAFKA_CONSUMER)) return -1; diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index ffa6a9d524..2cdcd7aa3c 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -557,8 +557,10 @@ rd_kafka_offset_broker_commit(rd_kafka_toppar_t *rktp, const char *reason) { 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); + rd_kafka_topic_partition_set_from_fetch_pos(rktpar, rktp->rktp_committing_pos); + rd_kafka_topic_partition_set_metadata_from_rktp_stored(rktpar, rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSETCMT", "%.*s [%" PRId32 "]: committing %s: %s", @@ -643,7 +645,8 @@ 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*/}; + rd_kafka_fetch_pos_t pos = + RD_KAFKA_FETCH_POS(offset + 1, -1 /*no leader epoch known*/); /* Find toppar */ rd_kafka_topic_rdlock(rkt); @@ -653,8 +656,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, pos, rd_false /* Don't force */, - RD_DO_LOCK); + err = rd_kafka_offset_store0(rktp, pos, NULL, 0, + rd_false /* Don't force */, RD_DO_LOCK); rd_kafka_toppar_destroy(rktp); @@ -675,7 +678,8 @@ 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}; + rd_kafka_fetch_pos_t pos = + RD_KAFKA_FETCH_POS(rktpar->offset, -1); rktp = rd_kafka_topic_partition_get_toppar(rk, rktpar, rd_false); @@ -689,7 +693,8 @@ rd_kafka_offsets_store(rd_kafka_t *rk, rd_kafka_topic_partition_get_leader_epoch(rktpar); rktpar->err = rd_kafka_offset_store0( - rktp, pos, rd_false /* don't force */, RD_DO_LOCK); + rktp, pos, rktpar->metadata, rktpar->metadata_size, + rd_false /* don't force */, RD_DO_LOCK); rd_kafka_toppar_destroy(rktp); if (rktpar->err) @@ -721,10 +726,10 @@ rd_kafka_error_t *rd_kafka_offset_store_message(rd_kafka_message_t *rkmessage) { "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); + pos = RD_KAFKA_FETCH_POS(rkmessage->offset + 1, + rkm->rkm_u.consumer.leader_epoch); + err = rd_kafka_offset_store0(rktp, pos, NULL, 0, + 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"); @@ -956,9 +961,6 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, "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; @@ -1020,17 +1022,18 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, if (end_offset < 0 || end_offset_leader_epoch < 0) { rd_kafka_offset_reset( - rktp, rd_kafka_broker_id(rkb), rktp->rktp_next_fetch_start, + rktp, rd_kafka_broker_id(rkb), + rktp->rktp_offset_validation_pos, RD_KAFKA_RESP_ERR__LOG_TRUNCATION, "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), + rd_kafka_fetch_pos2str(rktp->rktp_offset_validation_pos), end_offset, end_offset_leader_epoch); - } else if (end_offset < rktp->rktp_next_fetch_start.offset) { + } else if (end_offset < rktp->rktp_offset_validation_pos.offset) { if (rktp->rktp_rkt->rkt_conf.auto_offset_reset == RD_KAFKA_OFFSET_INVALID /* auto.offset.reset=error */) { @@ -1044,7 +1047,8 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, " (offset leader epoch %" PRId32 "). " "Reset to INVALID.", - rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), + rd_kafka_fetch_pos2str( + rktp->rktp_offset_validation_pos), end_offset, end_offset_leader_epoch); } else { @@ -1074,8 +1078,6 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk, rktp->rktp_partition, end_offset, end_offset_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); } @@ -1166,7 +1168,7 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { * 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) { + if (rktp->rktp_offset_validation_pos.leader_epoch == -1) { rd_kafka_dbg( rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE", "%.*s [%" PRId32 @@ -1174,7 +1176,7 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { "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)); + rd_kafka_fetch_pos2str(rktp->rktp_offset_validation_pos)); rd_kafka_toppar_set_fetch_state(rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE); return; @@ -1188,18 +1190,18 @@ void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) { 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); + rktpar, rktp->rktp_offset_validation_pos.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", - "%.*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_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_offset_validation_pos), reason); rd_kafka_OffsetForLeaderEpochRequest( rktp->rktp_leader, parts, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0), @@ -1441,7 +1443,7 @@ rd_kafka_resp_err_t rd_kafka_offset_store_stop(rd_kafka_toppar_t *rktp) { rktp, RD_KAFKA_FETCH_POS(rktp->rktp_offsets_fin.eof_offset, rktp->rktp_leader_epoch), - rd_true /* force */, RD_DONT_LOCK); + NULL, 0, rd_true /* force */, RD_DONT_LOCK); /* Commit offset to backing store. * This might be an async operation. */ @@ -1539,7 +1541,7 @@ void rd_kafka_update_app_pos(rd_kafka_t *rk, rktp->rktp_app_pos = pos; if (rk->rk_conf.enable_auto_offset_store) - rd_kafka_offset_store0(rktp, pos, + rd_kafka_offset_store0(rktp, pos, NULL, 0, /* force: ignore assignment state */ rd_true, RD_DONT_LOCK); diff --git a/src/rdkafka_offset.h b/src/rdkafka_offset.h index 7b01c84877..ba449995de 100644 --- a/src/rdkafka_offset.h +++ b/src/rdkafka_offset.h @@ -73,6 +73,8 @@ const char *rd_kafka_offset2str(int64_t offset); * * @param pos Offset and leader epoch to set, may be an absolute offset * or .._INVALID. + * @param metadata Metadata to be set (optional). + * @param metadata_size Size of the metadata to be set. * @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). * @@ -84,6 +86,8 @@ 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, const rd_kafka_fetch_pos_t pos, + void *metadata, + size_t metadata_size, rd_bool_t force, rd_dolock_t do_lock) { rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; @@ -96,7 +100,17 @@ rd_kafka_offset_store0(rd_kafka_toppar_t *rktp, !rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk))) { err = RD_KAFKA_RESP_ERR__STATE; } else { - rktp->rktp_stored_pos = pos; + if (rktp->rktp_stored_metadata) { + rd_free(rktp->rktp_stored_metadata); + rktp->rktp_stored_metadata = NULL; + } + rktp->rktp_stored_pos = pos; + rktp->rktp_stored_metadata_size = metadata_size; + if (metadata) { + rktp->rktp_stored_metadata = rd_malloc(metadata_size); + memcpy(rktp->rktp_stored_metadata, metadata, + rktp->rktp_stored_metadata_size); + } } if (do_lock) diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index b1f8774089..017b97d8b6 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -378,6 +378,8 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_METADATA: RD_IF_FREE(rko->rko_u.metadata.md, rd_kafka_metadata_destroy); + /* It's not needed to free metadata.mdi because they + are the in the same memory allocation. */ break; case RD_KAFKA_OP_LOG: diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 52153ed7af..73dee13ca4 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -378,9 +378,7 @@ struct rd_kafka_op_s { /* RD_KAFKA_OP_METADATA */ struct { rd_kafka_metadata_t *md; - rd_kafka_topic_authorized_operations_pair_t - *topic_authorized_operations; - int32_t cluster_authorized_operations; + rd_kafka_metadata_internal_t *mdi; int force; /* force request regardless of outstanding * metadata requests. */ } metadata; diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index 46d2fb3ed8..fcf6ae2f07 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -243,6 +243,7 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, 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_offset_validation_pos); 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); @@ -252,7 +253,7 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, mtx_init(&rktp->rktp_lock, mtx_plain); rd_refcnt_init(&rktp->rktp_refcnt, 0); - rktp->rktp_fetchq = rd_kafka_q_new(rkt->rkt_rk); + rktp->rktp_fetchq = rd_kafka_consume_q_new(rkt->rkt_rk); rktp->rktp_ops = rd_kafka_q_new(rkt->rkt_rk); rktp->rktp_ops->rkq_serve = rd_kafka_toppar_op_serve; rktp->rktp_ops->rkq_opaque = rktp; @@ -348,6 +349,7 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp) { rd_refcnt_destroy(&rktp->rktp_refcnt); + rd_free(rktp->rktp_stored_metadata); rd_free(rktp); } @@ -359,9 +361,6 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp) { * @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, - thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)); - if ((int)rktp->rktp_fetch_state == fetch_state) return; @@ -1798,6 +1797,7 @@ void rd_kafka_toppar_seek(rd_kafka_toppar_t *rktp, 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_toppar_set_offset_validation_position(rktp, pos); rd_kafka_offset_validate(rktp, "seek"); } @@ -2698,6 +2698,21 @@ void rd_kafka_topic_partition_set_from_fetch_pos( fetchpos.leader_epoch); } +/** + * @brief Set partition metadata from rktp stored one. + */ +void rd_kafka_topic_partition_set_metadata_from_rktp_stored( + rd_kafka_topic_partition_t *rktpar, + const rd_kafka_toppar_t *rktp) { + rktpar->metadata_size = rktp->rktp_stored_metadata_size; + if (rktp->rktp_stored_metadata) { + rktpar->metadata = rd_malloc(rktp->rktp_stored_metadata_size); + memcpy(rktpar->metadata, rktp->rktp_stored_metadata, + rktpar->metadata_size); + } +} + + /** * @brief Destroy all partitions in list. * @@ -3213,6 +3228,8 @@ int rd_kafka_topic_partition_list_set_offsets( verb = "setting stored"; rd_kafka_topic_partition_set_from_fetch_pos( rktpar, rktp->rktp_stored_pos); + rd_kafka_topic_partition_set_metadata_from_rktp_stored( + rktpar, rktp); } else { rktpar->offset = RD_KAFKA_OFFSET_INVALID; } diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 77a56f8049..16e9a86496 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -326,6 +326,10 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ * @locality toppar thread */ rd_kafka_fetch_pos_t rktp_last_next_fetch_start; + /** The offset to verify. + * @locality toppar thread */ + rd_kafka_fetch_pos_t rktp_offset_validation_pos; + /** Application's position. * This is the latest offset delivered to application + 1. * It is reset to INVALID_OFFSET when partition is @@ -335,6 +339,11 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ /** Last stored offset, but maybe not yet committed. */ rd_kafka_fetch_pos_t rktp_stored_pos; + /* Last stored metadata, but + * maybe not committed yet. */ + void *rktp_stored_metadata; + size_t rktp_stored_metadata_size; + /** Offset currently being committed */ rd_kafka_fetch_pos_t rktp_committing_pos; @@ -889,6 +898,10 @@ void rd_kafka_topic_partition_set_from_fetch_pos( rd_kafka_topic_partition_t *rktpar, const rd_kafka_fetch_pos_t fetchpos); +void rd_kafka_topic_partition_set_metadata_from_rktp_stored( + rd_kafka_topic_partition_t *rktpar, + const rd_kafka_toppar_t *rktp); + 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 = { @@ -1048,7 +1061,7 @@ 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. + * @locks 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, @@ -1056,4 +1069,15 @@ rd_kafka_toppar_set_next_fetch_position(rd_kafka_toppar_t *rktp, rktp->rktp_next_fetch_start = next_pos; } +/** + * @brief Sets the offset validation position. + * + * @locks rd_kafka_toppar_lock(rktp) MUST be held. + */ +static RD_UNUSED RD_INLINE void rd_kafka_toppar_set_offset_validation_position( + rd_kafka_toppar_t *rktp, + rd_kafka_fetch_pos_t offset_validation_pos) { + rktp->rktp_offset_validation_pos = offset_validation_pos; +} + #endif /* _RDKAFKA_PARTITION_H_ */ diff --git a/src/rdkafka_queue.c b/src/rdkafka_queue.c index 57fce36b8d..59a751abd9 100644 --- a/src/rdkafka_queue.c +++ b/src/rdkafka_queue.c @@ -83,12 +83,15 @@ void rd_kafka_q_destroy_final(rd_kafka_q_t *rkq) { */ void rd_kafka_q_init0(rd_kafka_q_t *rkq, rd_kafka_t *rk, + rd_bool_t for_consume, const char *func, int line) { rd_kafka_q_reset(rkq); rkq->rkq_fwdq = NULL; rkq->rkq_refcnt = 1; rkq->rkq_flags = RD_KAFKA_Q_F_READY; + if (for_consume) + rkq->rkq_flags |= RD_KAFKA_Q_F_CONSUMER; rkq->rkq_rk = rk; rkq->rkq_qio = NULL; rkq->rkq_serve = NULL; @@ -106,9 +109,15 @@ void rd_kafka_q_init0(rd_kafka_q_t *rkq, /** * Allocate a new queue and initialize it. */ -rd_kafka_q_t *rd_kafka_q_new0(rd_kafka_t *rk, const char *func, int line) { +rd_kafka_q_t *rd_kafka_q_new0(rd_kafka_t *rk, + rd_bool_t for_consume, + const char *func, + int line) { rd_kafka_q_t *rkq = rd_malloc(sizeof(*rkq)); - rd_kafka_q_init(rkq, rk); + if (!for_consume) + rd_kafka_q_init(rkq, rk); + else + rd_kafka_consume_q_init(rkq, rk); rkq->rkq_flags |= RD_KAFKA_Q_F_ALLOCATED; #if ENABLE_DEVEL rd_snprintf(rkq->rkq_name, sizeof(rkq->rkq_name), "%s:%d", func, line); @@ -118,6 +127,33 @@ rd_kafka_q_t *rd_kafka_q_new0(rd_kafka_t *rk, const char *func, int line) { return rkq; } +/* + * Sets the flag RD_KAFKA_Q_F_CONSUMER for rkq, any queues it's being forwarded + * to, recursively. + * Setting this flag indicates that polling this queue is equivalent to calling + * consumer poll, and will reset the max.poll.interval.ms timer. Only used + * internally when forwarding queues. + * @locks rd_kafka_q_lock(rkq) + */ +static void rd_kafka_q_consumer_propagate(rd_kafka_q_t *rkq) { + mtx_lock(&rkq->rkq_lock); + rkq->rkq_flags |= RD_KAFKA_Q_F_CONSUMER; + + if (!rkq->rkq_fwdq) { + mtx_unlock(&rkq->rkq_lock); + return; + } + + /* Recursively propagate the flag to any queues rkq is already + * forwarding to. There will be a deadlock here if the queues are being + * forwarded circularly, but that is a user error. We can't resolve this + * deadlock by unlocking before the recursive call, because that leads + * to incorrectness if the rkq_fwdq is forwarded elsewhere and the old + * one destroyed between recursive calls. */ + rd_kafka_q_consumer_propagate(rkq->rkq_fwdq); + mtx_unlock(&rkq->rkq_lock); +} + /** * Set/clear forward queue. * Queue forwarding enables message routing inside rdkafka. @@ -152,6 +188,9 @@ void rd_kafka_q_fwd_set0(rd_kafka_q_t *srcq, } srcq->rkq_fwdq = destq; + + if (srcq->rkq_flags & RD_KAFKA_Q_F_CONSUMER) + rd_kafka_q_consumer_propagate(destq); } if (do_lock) mtx_unlock(&srcq->rkq_lock); @@ -610,6 +649,7 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq, rd_kafka_q_destroy(fwdq); return cnt; } + mtx_unlock(&rkq->rkq_lock); if (timeout_ms) @@ -684,7 +724,7 @@ 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_pos.offset < offset)) + if (unlikely(rktp && (rktp->rktp_app_pos.offset < offset))) rd_kafka_update_app_pos( rk, rktp, RD_KAFKA_FETCH_POS( @@ -708,7 +748,7 @@ 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_pos.offset < offset) + if (rktp && (rktp->rktp_app_pos.offset < offset)) rd_kafka_update_app_pos( rk, rktp, RD_KAFKA_FETCH_POS( diff --git a/src/rdkafka_queue.h b/src/rdkafka_queue.h index 0d50f58703..82abe4deef 100644 --- a/src/rdkafka_queue.h +++ b/src/rdkafka_queue.h @@ -75,6 +75,11 @@ struct rd_kafka_q_s { * by triggering the cond-var \ * but without having to enqueue \ * an op. */ +#define RD_KAFKA_Q_F_CONSUMER \ + 0x10 /* If this flag is set, this queue might contain fetched messages \ + from partitions. Polling this queue will reset the \ + max.poll.interval.ms timer. Once set, this flag is never \ + reset. */ rd_kafka_t *rkq_rk; struct rd_kafka_q_io *rkq_qio; /* FD-based application signalling */ @@ -123,12 +128,20 @@ static RD_INLINE RD_UNUSED int rd_kafka_q_ready(rd_kafka_q_t *rkq) { void rd_kafka_q_init0(rd_kafka_q_t *rkq, rd_kafka_t *rk, + rd_bool_t for_consume, const char *func, int line); #define rd_kafka_q_init(rkq, rk) \ - rd_kafka_q_init0(rkq, rk, __FUNCTION__, __LINE__) -rd_kafka_q_t *rd_kafka_q_new0(rd_kafka_t *rk, const char *func, int line); -#define rd_kafka_q_new(rk) rd_kafka_q_new0(rk, __FUNCTION__, __LINE__) + rd_kafka_q_init0(rkq, rk, rd_false, __FUNCTION__, __LINE__) +#define rd_kafka_consume_q_init(rkq, rk) \ + rd_kafka_q_init0(rkq, rk, rd_true, __FUNCTION__, __LINE__) +rd_kafka_q_t *rd_kafka_q_new0(rd_kafka_t *rk, + rd_bool_t for_consume, + const char *func, + int line); +#define rd_kafka_q_new(rk) rd_kafka_q_new0(rk, rd_false, __FUNCTION__, __LINE__) +#define rd_kafka_consume_q_new(rk) \ + rd_kafka_q_new0(rk, rd_true, __FUNCTION__, __LINE__) void rd_kafka_q_destroy_final(rd_kafka_q_t *rkq); #define rd_kafka_q_lock(rkqu) mtx_lock(&(rkqu)->rkq_lock) @@ -1164,6 +1177,22 @@ rd_kafka_enq_once_disable(rd_kafka_enq_once_t *eonce) { return rko; } +/** + * @brief Returns true if the queue can contain fetched messages. + * + * @locks rd_kafka_q_lock(rkq) if do_lock is set. + */ +static RD_INLINE RD_UNUSED rd_bool_t +rd_kafka_q_can_contain_fetched_msgs(rd_kafka_q_t *rkq, rd_bool_t do_lock) { + rd_bool_t val; + if (do_lock) + mtx_lock(&rkq->rkq_lock); + val = rkq->rkq_flags & RD_KAFKA_Q_F_CONSUMER; + if (do_lock) + mtx_unlock(&rkq->rkq_lock); + return val; +} + /**@}*/ diff --git a/src/rdkafka_range_assignor.c b/src/rdkafka_range_assignor.c index c83f1f1a44..60f7c1e112 100644 --- a/src/rdkafka_range_assignor.c +++ b/src/rdkafka_range_assignor.c @@ -27,7 +27,7 @@ */ #include "rdkafka_int.h" #include "rdkafka_assignor.h" - +#include "rdunittest.h" /** @@ -50,6 +50,445 @@ * C1: [t0p2, t1p2] */ +typedef struct { + rd_kafkap_str_t *member_id; + rd_list_t *assigned_partitions; /* Contained Type: int* */ +} rd_kafka_member_assigned_partitions_pair_t; + +/** + * @brief Intializes a rd_kafka_member_assigned_partitions_pair_t* with + * assigned_partitions = []. + * + * @param member_id + * + * The member_id isn't copied, so the returned value can be used only for the + * lifetime of this function's arguments. + * @return rd_kafka_member_assigned_partitions_pair_t* + */ +static rd_kafka_member_assigned_partitions_pair_t * +rd_kafka_member_assigned_partitions_pair_new(rd_kafkap_str_t *member_id) { + rd_kafka_member_assigned_partitions_pair_t *pair = + rd_calloc(1, sizeof(rd_kafka_member_assigned_partitions_pair_t)); + + pair->member_id = member_id; + pair->assigned_partitions = rd_list_new(0, NULL); + return pair; +} + +static void rd_kafka_member_assigned_partitions_pair_destroy(void *_pair) { + rd_kafka_member_assigned_partitions_pair_t *pair = + (rd_kafka_member_assigned_partitions_pair_t *)_pair; + + /* Do not destroy the member_id, we don't take ownership. */ + RD_IF_FREE(pair->assigned_partitions, rd_list_destroy); + RD_IF_FREE(pair, rd_free); +} + +static int rd_kafka_member_assigned_partitions_pair_cmp(const void *_a, + const void *_b) { + rd_kafka_member_assigned_partitions_pair_t *a = + (rd_kafka_member_assigned_partitions_pair_t *)_a; + rd_kafka_member_assigned_partitions_pair_t *b = + (rd_kafka_member_assigned_partitions_pair_t *)_b; + return rd_kafkap_str_cmp(a->member_id, b->member_id); +} + +static rd_kafka_member_assigned_partitions_pair_t * +rd_kafka_find_member_assigned_partitions_pair_by_member_id( + rd_kafkap_str_t *member_id, + rd_list_t *rd_kafka_member_assigned_partitions_pair_list) { + rd_kafka_member_assigned_partitions_pair_t search_pair = {member_id, + NULL}; + return rd_list_find(rd_kafka_member_assigned_partitions_pair_list, + &search_pair, + rd_kafka_member_assigned_partitions_pair_cmp); +} + +typedef struct { + /* Contains topic and list of members - sorted by group instance id and + * member id. Also contains partitions, along with partition replicas, + * which will help us with the racks. The members also contain their + * rack id and the partitions they have already been assigned. + */ + rd_kafka_assignor_topic_t *topic; + /* unassigned_partitions[i] is true if the ith partition of this topic + * is not assigned. We prefer using an array rather than using an + * rd_list and removing elements, because that involves a memmove on + * each remove. */ + rd_bool_t *unassigned_partitions; + /* Number of partitions still to be assigned.*/ + size_t unassigned_partitions_left; + /* An array of char** arrays. The ith element of this array is a sorted + * char** array, denoting the racks for the ith partition of this topic. + * The size of this array is equal to the partition_cnt. */ + char ***partition_racks; + /* The ith element of this array is the size of partition_racks[i]. */ + size_t *racks_cnt; + /* Contains a pair denoting the partitions assigned to every subscribed + * consumer (member, [rd_list_t* of int*]). Sorted by member_id. + * Contained Type: rd_kafka_member_assigned_partitions_pair_t* */ + rd_list_t *member_to_assigned_partitions; + /* Contains the number of partitions that should be ideally assigned to + * every subscribing consumer. */ + int num_partitions_per_consumer; + /* Contains the number of consumers with extra partitions in case number + * of partitions isn't perfectly divisible by number of consumers. */ + int remaining_consumers_with_extra_partition; + /* True if we need to perform rack aware assignment. */ + rd_bool_t needs_rack_aware_assignment; +} rd_kafka_topic_assignment_state_t; + + +/** + * @brief Initialize an rd_kafka_topic_assignment_state_t. + * + * @param topic + * @param broker_rack_pair + * @param broker_rack_pair_cnt + * + * The struct rd_kafka_topic_assignment_state_t is mostly for convenience and + * easy grouping, so we avoid copying values as much as possible. Hence, the + * returned rd_kafka_topic_assignment_state_t does not own all its values, and + * should not be used beyond the lifetime of this function's arguments. This + * function also computes the value of needsRackAwareAssignment given the other + * information. + * + * @return rd_kafka_topic_assignment_state_t* + */ + +static rd_kafka_topic_assignment_state_t * +rd_kafka_topic_assignment_state_new(rd_kafka_assignor_topic_t *topic, + const rd_kafka_metadata_internal_t *mdi) { + int i; + rd_kafka_group_member_t *member; + rd_kafka_topic_assignment_state_t *rktas; + const int partition_cnt = topic->metadata->partition_cnt; + + rktas = rd_calloc(1, sizeof(rd_kafka_topic_assignment_state_t)); + rktas->topic = topic; /* don't copy. */ + + rktas->unassigned_partitions = + rd_malloc(sizeof(rd_bool_t) * partition_cnt); + rktas->unassigned_partitions_left = partition_cnt; + for (i = 0; i < partition_cnt; i++) { + rktas->unassigned_partitions[i] = rd_true; + } + + rktas->num_partitions_per_consumer = 0; + rktas->remaining_consumers_with_extra_partition = 0; + if (rd_list_cnt(&topic->members)) { + rktas->num_partitions_per_consumer = + partition_cnt / rd_list_cnt(&topic->members); + rktas->remaining_consumers_with_extra_partition = + partition_cnt % rd_list_cnt(&topic->members); + } + + rktas->member_to_assigned_partitions = + rd_list_new(0, rd_kafka_member_assigned_partitions_pair_destroy); + + RD_LIST_FOREACH(member, &topic->members, i) { + rd_list_add(rktas->member_to_assigned_partitions, + rd_kafka_member_assigned_partitions_pair_new( + member->rkgm_member_id)); + } + + rd_list_sort(rktas->member_to_assigned_partitions, + rd_kafka_member_assigned_partitions_pair_cmp); + + rktas->partition_racks = rd_calloc(partition_cnt, sizeof(char **)); + rktas->racks_cnt = rd_calloc(partition_cnt, sizeof(size_t)); + for (i = 0; topic->metadata_internal->partitions && i < partition_cnt; + i++) { + rktas->racks_cnt[i] = + topic->metadata_internal->partitions[i].racks_cnt; + rktas->partition_racks[i] = + topic->metadata_internal->partitions[i].racks; + } + + rktas->needs_rack_aware_assignment = + rd_kafka_use_rack_aware_assignment(&topic, 1, mdi); + + return rktas; +} + +/* Destroy a rd_kafka_topic_assignment_state_t. */ +static void rd_kafka_topic_assignment_state_destroy(void *_rktas) { + rd_kafka_topic_assignment_state_t *rktas = + (rd_kafka_topic_assignment_state_t *)_rktas; + + rd_free(rktas->unassigned_partitions); + rd_list_destroy(rktas->member_to_assigned_partitions); + rd_free(rktas->partition_racks); + rd_free(rktas->racks_cnt); + rd_free(rktas); +} + +/** + * Compare two topic_assignment_states, first on the sorted list of consumers + * (each consumer from the list of consumers is matched till the first point of + * difference), and if that's equal, compare on the number of partitions. + * + * A list sorted with this comparator will group the topic_assignment_states + * having the same consumers and the same number of partitions together - this + * is the criteria of co-partitioned topics. + */ +static int rd_kafka_topic_assignment_state_cmp(const void *_a, const void *_b) { + int i; + rd_kafka_topic_assignment_state_t *a = + (rd_kafka_topic_assignment_state_t *)_a; + rd_kafka_topic_assignment_state_t *b = + (rd_kafka_topic_assignment_state_t *)_b; + + /* This guarantee comes from rd_kafka_range_assignor_assign_cb. */ + rd_assert(a->topic->members.rl_flags & RD_LIST_F_SORTED); + rd_assert(b->topic->members.rl_flags & RD_LIST_F_SORTED); + + /* Based on consumers */ + for (i = 0; i < rd_list_cnt(&a->topic->members) && + i < rd_list_cnt(&b->topic->members); + i++) { + rd_kafka_group_member_t *am = + rd_list_elem(&a->topic->members, i); + rd_kafka_group_member_t *bm = + rd_list_elem(&b->topic->members, i); + int cmp_res = + rd_kafkap_str_cmp(am->rkgm_member_id, bm->rkgm_member_id); + if (cmp_res != 0) + return cmp_res; + } + + if (rd_list_cnt(&a->topic->members) != + rd_list_cnt(&b->topic->members)) { + return RD_CMP(rd_list_cnt(&a->topic->members), + rd_list_cnt(&b->topic->members)); + } + + /* Based on number of partitions */ + return RD_CMP(a->topic->metadata->partition_cnt, + b->topic->metadata->partition_cnt); +} + + +/* Helper function to wrap a bsearch on the partition's racks. */ +static char *rd_kafka_topic_assignment_state_rack_search( + rd_kafka_topic_assignment_state_t *rktas, + int partition, + const char *rack) { + char **partition_racks = rktas->partition_racks[partition]; + size_t cnt = rktas->racks_cnt[partition]; + void *res = NULL; + + if (!partition_racks) + return NULL; + + res = bsearch(&rack, partition_racks, cnt, sizeof(char *), rd_strcmp3); + if (!res) + return NULL; + + return *(char **)res; +} + +/* + * Assigns a partition to a member, and updates fields in rktas for accounting. + * It's assumed that the partitions assigned to this member don't exceed the + * allowed number. + */ +static void rd_kafka_assign_partition(rd_kafka_group_member_t *member, + rd_kafka_topic_assignment_state_t *rktas, + int32_t partition) { + rd_kafka_member_assigned_partitions_pair_t *member_assignment = + rd_kafka_find_member_assigned_partitions_pair_by_member_id( + member->rkgm_member_id, rktas->member_to_assigned_partitions); + rd_assert(member_assignment); + + /* We can't use &partition, since that's a copy on the stack. */ + rd_list_add(member_assignment->assigned_partitions, + (void *)&rktas->topic->metadata->partitions[partition].id); + rd_kafka_topic_partition_list_add_range(member->rkgm_assignment, + rktas->topic->metadata->topic, + partition, partition); + + rd_assert(rktas->unassigned_partitions[partition]); + rktas->unassigned_partitions[partition] = rd_false; + rktas->unassigned_partitions_left--; + + if (rd_list_cnt(member_assignment->assigned_partitions) > + rktas->num_partitions_per_consumer) { + rktas->remaining_consumers_with_extra_partition -= 1; + } +} + + +/* Implementation of may_assign for rd_kafka_assign_ranges. True if the consumer + * rack is empty, or if is exists within the partition racks. */ +static rd_bool_t rd_kafka_racks_match(rd_kafka_group_member_t *member, + rd_kafka_topic_assignment_state_t *rktas, + int32_t partition) { + rd_kafkap_str_t *consumer_rack = member->rkgm_rack_id; + + if (!consumer_rack || RD_KAFKAP_STR_LEN(consumer_rack) == 0) { + return rd_true; + } + + return rd_kafka_topic_assignment_state_rack_search( + rktas, partition, consumer_rack->str) != NULL; +} + + +/* Implementation of may_assign for rd_kafka_assign_ranges. Always true, used to + * assign remaining partitions after rack-aware assignment is complete. */ +static rd_bool_t rd_kafka_always(rd_kafka_group_member_t *member, + rd_kafka_topic_assignment_state_t *rktas, + int32_t partition) { + return rd_true; +} + +/* Assigns as many partitions as possible for a topic to subscribing members, + * such that no subscribing member exceeds their limit of allowed partitions, + * and may_assign(member, rktas, partition) is true for each member and + * partition. + */ +static void rd_kafka_assign_ranges( + rd_kafka_topic_assignment_state_t *rktas, + rd_bool_t (*may_assign)(rd_kafka_group_member_t *member, + rd_kafka_topic_assignment_state_t *rktas, + int32_t partition)) { + int i; + rd_kafka_group_member_t *member; + int32_t *partitions_to_assign = + rd_alloca(rktas->unassigned_partitions_left * sizeof(int32_t)); + + RD_LIST_FOREACH(member, &rktas->topic->members, i) { + int j; + rd_kafka_member_assigned_partitions_pair_t *member_assignment; + int maximum_assignable_to_consumer; + int partitions_to_assign_cnt; + + if (rktas->unassigned_partitions_left == 0) + break; + + member_assignment = + rd_kafka_find_member_assigned_partitions_pair_by_member_id( + member->rkgm_member_id, + rktas->member_to_assigned_partitions); + + maximum_assignable_to_consumer = + rktas->num_partitions_per_consumer + + (rktas->remaining_consumers_with_extra_partition > 0) - + rd_list_cnt(member_assignment->assigned_partitions); + + if (maximum_assignable_to_consumer <= 0) + continue; + + partitions_to_assign_cnt = 0; + for (j = 0; j < rktas->topic->metadata->partition_cnt; j++) { + if (!rktas->unassigned_partitions[j]) { + continue; + } + + if (maximum_assignable_to_consumer <= 0) + break; + + if (!may_assign(member, rktas, j)) + continue; + + partitions_to_assign[partitions_to_assign_cnt] = j; + partitions_to_assign_cnt++; + maximum_assignable_to_consumer--; + } + + for (j = 0; j < partitions_to_assign_cnt; j++) + rd_kafka_assign_partition(member, rktas, + partitions_to_assign[j]); + } +} + +/* + * Assigns partitions for co-partitioned topics in a rack-aware manner on a best + * effort basis. All partitions may not be assigned to consumers in case a rack + * aware assignment does not exist. + */ +static void rd_kafka_assign_co_partitioned( + rd_list_t * + rktas_bucket /* Contained Type: rd_kafka_topic_assignment_state_t* */) { + rd_kafka_topic_assignment_state_t *first_rktas = + rd_list_elem(rktas_bucket, 0); + rd_kafka_topic_assignment_state_t *rktas; + rd_kafka_group_member_t *member; + int i; + + /* Since a "bucket" is a group of topic_assignment_states with the same + * consumers and number of partitions, we can just fetch them from the + * first member of the bucket. */ + const int partition_cnt = first_rktas->topic->metadata->partition_cnt; + const rd_list_t *consumers = &first_rktas->topic->members; + + for (i = 0; i < partition_cnt; i++) { + /* + * To assign the ith partition of all the co partitioned topics, + * we need to find a consumerX that fulfils the criteria: + * for all topic_assignment_states in the bucket: + * 1. rack(consumerX) is contained inside racks(partition i) + * 2. partitions assigned to consumerX does not exceed limits. + */ + int j; + RD_LIST_FOREACH(member, consumers, j) { + int m; + RD_LIST_FOREACH(rktas, rktas_bucket, m) { + int maximum_assignable; + rd_kafka_member_assigned_partitions_pair_t + *member_assignment; + + /* Check (1.) */ + if (!member->rkgm_rack_id || + RD_KAFKAP_STR_LEN(member->rkgm_rack_id) == + 0 || + rd_kafka_topic_assignment_state_rack_search( + rktas, i, member->rkgm_rack_id->str) == + NULL) { + break; + } + + /* Check (2.) */ + member_assignment = + rd_kafka_find_member_assigned_partitions_pair_by_member_id( + member->rkgm_member_id, + rktas->member_to_assigned_partitions); + maximum_assignable = + rktas->num_partitions_per_consumer + + (rktas + ->remaining_consumers_with_extra_partition > + 0) - + rd_list_cnt( + member_assignment->assigned_partitions); + + if (maximum_assignable <= 0) { + break; + } + } + if (m == rd_list_cnt(rktas_bucket)) { + /* Break early - this consumer can be assigned + * this partition. */ + break; + } + } + if (j == rd_list_cnt(&first_rktas->topic->members)) { + continue; /* We didn't find a suitable consumer. */ + } + + rd_assert(member); + + RD_LIST_FOREACH(rktas, rktas_bucket, j) { + rd_kafka_assign_partition(member, rktas, i); + } + + /* FIXME: A possible optimization: early break here if no + * consumer remains with maximum_assignable_to_consumer > 0 + * across all topics. */ + } +} + + rd_kafka_resp_err_t rd_kafka_range_assignor_assign_cb(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas, @@ -64,67 +503,1236 @@ rd_kafka_range_assignor_assign_cb(rd_kafka_t *rk, void *opaque) { unsigned int ti; int i; + rd_list_t *rktas_list = rd_list_new( + eligible_topic_cnt, rd_kafka_topic_assignment_state_destroy); + rd_list_t *rktas_buckets = rd_list_new(0, rd_list_destroy_free); + rd_list_t + *rktas_current_bucket; /* Contained Type: + rd_kafka_topic_assignment_state_t* */ + rd_kafka_topic_assignment_state_t *rktas; + rd_kafka_topic_assignment_state_t *prev_rktas; + const rd_kafka_metadata_internal_t *mdi = + rd_kafka_metadata_get_internal(metadata); /* The range assignor works on a per-topic basis. */ for (ti = 0; ti < eligible_topic_cnt; ti++) { rd_kafka_assignor_topic_t *eligible_topic = eligible_topics[ti]; - int numPartitionsPerConsumer; - int consumersWithExtraPartition; - /* For each topic, we lay out the available partitions in - * numeric order and the consumers in lexicographic order. */ + /* For each topic, we sort the consumers in lexicographic order, + * and create a topic_assignment_state. */ rd_list_sort(&eligible_topic->members, rd_kafka_group_member_cmp); + rd_list_add(rktas_list, rd_kafka_topic_assignment_state_new( + eligible_topic, mdi)); + } - /* We then divide the number of partitions by the total number - * of consumers to determine the number of partitions to assign - * to each consumer. */ - numPartitionsPerConsumer = - eligible_topic->metadata->partition_cnt / - rd_list_cnt(&eligible_topic->members); + /* Sort the topic_assignment_states to group the topics which need to be + * co-partitioned. */ + rd_list_sort(rktas_list, rd_kafka_topic_assignment_state_cmp); - /* If it does not evenly divide, then the first few consumers - * will have one extra partition. */ - consumersWithExtraPartition = - eligible_topic->metadata->partition_cnt % - rd_list_cnt(&eligible_topic->members); + /* Use the sorted list of topic_assignment_states and separate them into + * "buckets". Each bucket contains topics which can be co-partitioned, + * ie with the same consumers and number of partitions. */ + prev_rktas = NULL; + rktas_current_bucket = NULL; + RD_LIST_FOREACH(rktas, rktas_list, i) { + if (prev_rktas && rd_kafka_topic_assignment_state_cmp( + rktas, prev_rktas) == 0) { + rd_list_add(rktas_current_bucket, rktas); + continue; + } - rd_kafka_dbg(rk, CGRP, "ASSIGN", - "range: Topic %s with %d partition(s) and " - "%d subscribing member(s)", - eligible_topic->metadata->topic, - eligible_topic->metadata->partition_cnt, - rd_list_cnt(&eligible_topic->members)); - - for (i = 0; i < rd_list_cnt(&eligible_topic->members); i++) { - rd_kafka_group_member_t *rkgm = - rd_list_elem(&eligible_topic->members, i); - int start = numPartitionsPerConsumer * i + - RD_MIN(i, consumersWithExtraPartition); - int length = - numPartitionsPerConsumer + - (i + 1 > consumersWithExtraPartition ? 0 : 1); - - if (length == 0) + /* The free function is set to NULL, as we don't copy any of the + * topic_assignment_states. */ + rktas_current_bucket = rd_list_new(0, NULL); + rd_list_add(rktas_buckets, rktas_current_bucket); + prev_rktas = rktas; + rd_list_add(rktas_current_bucket, rktas); + } + + /* Iterate through each bucket. In case there's more than one element in + * the bucket, we prefer co-partitioning over rack awareness. Otherwise, + * assign with rack-awareness. */ + rktas = NULL; + rktas_current_bucket = NULL; + RD_LIST_FOREACH(rktas_current_bucket, rktas_buckets, i) { + rd_assert(rd_list_cnt(rktas_current_bucket) > 0); + + if (rd_list_cnt(rktas_current_bucket) == 1) { + rktas = rd_list_elem(rktas_current_bucket, 0); + if (!rktas->needs_rack_aware_assignment) continue; + rd_kafka_dbg(rk, CGRP, "ASSIGN", - "range: Member \"%s\": " - "assigned topic %s partitions %d..%d", - rkgm->rkgm_member_id->str, - eligible_topic->metadata->topic, start, - start + length - 1); - rd_kafka_topic_partition_list_add_range( - rkgm->rkgm_assignment, - eligible_topic->metadata->topic, start, - start + length - 1); + "range: Topic %s with %d partition(s) and " + "%d subscribing member(s), single-topic " + "rack-aware assignment", + rktas->topic->metadata->topic, + rktas->topic->metadata->partition_cnt, + rd_list_cnt(&rktas->topic->members)); + + rd_kafka_assign_ranges(rktas, rd_kafka_racks_match); + } else { + rktas = rd_list_elem(rktas_current_bucket, 0); + rd_kafka_dbg( + rk, CGRP, "ASSIGN", + "range: %d topics with %d partition(s) and " + "%d subscribing member(s), co-partitioned " + "rack-aware assignment", + rd_list_cnt(rktas_current_bucket), + rktas->topic->metadata->partition_cnt, + rd_list_cnt(&rktas->topic->members)); + + rd_kafka_assign_co_partitioned(rktas_current_bucket); } } + /* Iterate through each rktas, doing normal assignment for any + * partitions that might not have gotten a rack-aware assignment.*/ + RD_LIST_FOREACH(rktas, rktas_list, i) { + rd_kafka_dbg(rk, CGRP, "ASSIGN", + "range: Topic %s with %d partition(s) and " + "%d subscribing member(s), single-topic " + "non-rack-aware assignment for %" PRIusz + " leftover partitions", + rktas->topic->metadata->topic, + rktas->topic->metadata->partition_cnt, + rd_list_cnt(&rktas->topic->members), + rktas->unassigned_partitions_left); + rd_kafka_assign_ranges(rktas, rd_kafka_always); + } + + rd_list_destroy(rktas_list); + rd_list_destroy(rktas_buckets); + return 0; } +/** + * @name Sticky assignor unit tests + * + * + * These are based on RangeAssignorTest.java + * + * + * + */ + + +/* All possible racks used in tests, as well as several common rack configs used + * by consumers */ +static rd_kafkap_str_t + *ALL_RACKS[7]; /* initialized before starting the unit tests. */ +static int RACKS_INITIAL[] = {0, 1, 2}; +static int RACKS_NULL[] = {6, 6, 6}; +static int RACKS_FINAL[] = {4, 5, 6}; +static int RACKS_ONE_NULL[] = {6, 4, 5}; + +static int +ut_testOneConsumerNoTopic(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[1]; + + + if (parametrization == RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { + RD_UT_PASS(); + } + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 0); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], NULL); + + rd_kafka_group_member_clear(&members[0]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testOneConsumerNonexistentTopic( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[1]; + + + if (parametrization == RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { + RD_UT_PASS(); + } + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "t1", 0); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], NULL); + + rd_kafka_group_member_clear(&members[0]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int +ut_testOneConsumerOneTopic(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[1]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "t1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + RD_UT_ASSERT(members[0].rkgm_assignment->cnt == 3, + "expected assignment of 3 partitions, got %d partition(s)", + members[0].rkgm_assignment->cnt); + + verifyAssignment(&members[0], "t1", 0, "t1", 1, "t1", 2, NULL); + + rd_kafka_group_member_clear(&members[0]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int ut_testOnlyAssignsPartitionsFromSubscribedTopics( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[1]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "t1", 3, "t2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, "t1", 1, "t1", 2, NULL); + + rd_kafka_group_member_clear(&members[0]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testOneConsumerMultipleTopics( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[1]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "t1", 1, "t2", 2); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", "t2", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, "t2", 0, "t2", 1, NULL); + + rd_kafka_group_member_clear(&members[0]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testTwoConsumersOneTopicOnePartition( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "t1", 1); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", ALL_RACKS[1], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, NULL); + verifyAssignment(&members[1], NULL); + + rd_kafka_group_member_clear(&members[0]); + rd_kafka_group_member_clear(&members[1]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testTwoConsumersOneTopicTwoPartitions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "t1", 2); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", ALL_RACKS[1], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, NULL); + verifyAssignment(&members[1], "t1", 1, NULL); + + rd_kafka_group_member_clear(&members[0]); + rd_kafka_group_member_clear(&members[1]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testMultipleConsumersMixedTopicSubscriptions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "t1", 3, "t2", 2); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", ALL_RACKS[1], + parametrization, "t1", "t2", NULL); + ut_initMemberConditionalRack(&members[2], "consumer3", ALL_RACKS[2], + parametrization, "t1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, NULL); + verifyAssignment(&members[1], "t1", 1, "t2", 0, "t2", 1, NULL); + verifyAssignment(&members[2], "t1", 2, NULL); + + rd_kafka_group_member_clear(&members[0]); + rd_kafka_group_member_clear(&members[1]); + rd_kafka_group_member_clear(&members[2]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testTwoConsumersTwoTopicsSixPartitions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "t1", 3, "t2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", ALL_RACKS[0], + parametrization, "t1", "t2", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", ALL_RACKS[1], + parametrization, "t1", "t2", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + RD_ARRAYSIZE(members), errstr, + sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyAssignment(&members[0], "t1", 0, "t1", 1, "t2", 0, "t2", 1, NULL); + verifyAssignment(&members[1], "t1", 2, "t2", 2, NULL); + + rd_kafka_group_member_clear(&members[0]); + rd_kafka_group_member_clear(&members[1]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +/* Helper for setting up metadata and members, and running the assignor. Does + * not check the results of the assignment. */ +static int setupRackAwareAssignment0(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_group_member_t *members, + size_t member_cnt, + int replication_factor, + int num_broker_racks, + size_t topic_cnt, + char *topics[], + int *partitions, + int *subscriptions_count, + char **subscriptions[], + int *consumer_racks, + rd_kafka_metadata_t **metadata) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata_local = NULL; + if (!metadata) + metadata = &metadata_local; + + size_t i = 0; + const int num_brokers = num_broker_racks > 0 + ? replication_factor * num_broker_racks + : replication_factor; + + /* The member naming for tests is consumerN where N is a single + * character. */ + rd_assert(member_cnt <= 9); + + *metadata = rd_kafka_metadata_new_topic_with_partition_replicas_mock( + replication_factor, num_brokers, topics, partitions, topic_cnt); + ut_populate_internal_broker_metadata( + rd_kafka_metadata_get_internal(*metadata), num_broker_racks, + ALL_RACKS, RD_ARRAYSIZE(ALL_RACKS)); + ut_populate_internal_topic_metadata( + rd_kafka_metadata_get_internal(*metadata)); + + for (i = 0; i < member_cnt; i++) { + char member_id[10]; + snprintf(member_id, 10, "consumer%d", (int)(i + 1)); + ut_init_member_with_rack( + &members[i], member_id, ALL_RACKS[consumer_racks[i]], + subscriptions[i], subscriptions_count[i]); + } + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, *metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + if (metadata_local) + ut_destroy_metadata(metadata_local); + return 0; +} + +static int setupRackAwareAssignment(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_group_member_t *members, + size_t member_cnt, + int replication_factor, + int num_broker_racks, + size_t topic_cnt, + char *topics[], + int *partitions, + int *subscriptions_count, + char **subscriptions[], + int *consumer_racks) { + return setupRackAwareAssignment0( + rk, rkas, members, member_cnt, replication_factor, num_broker_racks, + topic_cnt, topics, partitions, subscriptions_count, subscriptions, + consumer_racks, NULL); +} + +/* Helper for testing cases where rack-aware assignment should not be triggered, + * and assignment should be the same as the pre-rack-aware assignor. */ +#define verifyNonRackAwareAssignment(rk, rkas, members, member_cnt, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, ...) \ + do { \ + size_t idx = 0; \ + rd_kafka_metadata_t *metadata = NULL; \ + \ + /* num_broker_racks = 0, implies that brokers have no \ + * configured racks. */ \ + setupRackAwareAssignment(rk, rkas, members, member_cnt, 3, 0, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_INITIAL); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* consumer_racks = RACKS_NULL implies that consumers have no \ + * racks. */ \ + setupRackAwareAssignment(rk, rkas, members, member_cnt, 3, 3, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_NULL); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* replication_factor = 3 and num_broker_racks = 3 means that \ + * all partitions are replicated on all racks.*/ \ + setupRackAwareAssignment0(rk, rkas, members, member_cnt, 3, 3, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_INITIAL, &metadata); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + verifyNumPartitionsWithRackMismatch(metadata, members, \ + RD_ARRAYSIZE(members), 0); \ + \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + ut_destroy_metadata(metadata); \ + /* replication_factor = 4 and num_broker_racks = 4 means that \ + * all partitions are replicated on all racks. */ \ + setupRackAwareAssignment0(rk, rkas, members, member_cnt, 4, 4, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_INITIAL, &metadata); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + verifyNumPartitionsWithRackMismatch(metadata, members, \ + RD_ARRAYSIZE(members), 0); \ + \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + ut_destroy_metadata(metadata); \ + /* There's no overap between broker racks and consumer racks, \ + * since num_broker_racks = 3, they'll be picked from a,b,c \ + * and consumer racks are d,e,f. */ \ + setupRackAwareAssignment(rk, rkas, members, member_cnt, 3, 3, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_FINAL); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* There's no overap between broker racks and consumer racks, \ + * since num_broker_racks = 3, they'll be picked from a,b,c \ + * and consumer racks are d,e,NULL. */ \ + setupRackAwareAssignment(rk, rkas, members, member_cnt, 3, 3, \ + topic_cnt, topics, partitions, \ + subscriptions_count, subscriptions, \ + RACKS_ONE_NULL); \ + verifyMultipleAssignment(members, member_cnt, __VA_ARGS__); \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + } while (0) + +static int ut_testRackAwareAssignmentWithUniformSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + char *topics[] = {"t1", "t2", "t3"}; + int partitions[] = {6, 7, 2}; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + size_t i = 0; + int subscriptions_count[] = {3, 3, 3}; + char **subscriptions[] = {topics, topics, topics}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, + /* consumer1*/ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 2, "t3", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 3, "t2", 4, "t3", 1, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, "t2", 5, "t2", 6, NULL); + + /* Verify best-effort rack-aware assignment for lower replication factor + * where racks have a subset of partitions.*/ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 2, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /*consumer1*/ + "t1", 0, "t1", 2, "t2", 0, "t2", 2, "t2", 3, "t3", 1, NULL, + /* consumer2 */ + "t1", 1, "t1", 3, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer 3*/ + "t1", 4, "t1", 5, "t2", 5, "t2", 6, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 1); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + + /* One consumer on a rack with no partitions. */ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 3, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment(members, RD_ARRAYSIZE(members), + /* consumer1 */ "t1", 0, "t1", 1, "t2", 0, + "t2", 1, "t2", 2, "t3", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 3, "t2", 4, "t3", 1, + NULL, + /* consumer3 */ + "t1", 4, "t1", 5, "t2", 5, "t2", 6, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 4); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testRackAwareAssignmentWithNonEqualSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_metadata_t *metadata; + char *topics[] = {"t1", "t2", "t3"}; + int partitions[] = {6, 7, 2}; + rd_kafka_group_member_t members[3]; + size_t i = 0; + int subscriptions_count[] = {3, 3, 2}; + char *subscription13[] = {"t1", "t3"}; + char **subscriptions[] = {topics, topics, subscription13}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, + /* consumer1*/ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 2, "t2", 3, "t3", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 4, "t2", 5, "t2", 6, "t3", 1, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, NULL); + + /* Verify best-effort rack-aware assignment for lower replication factor + * where racks have a subset of partitions. */ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 2, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t2", 5, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 2); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 2, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 2, "t2", 0, "t2", 2, "t2", 3, "t2", 5, "t3", 1, NULL, + /* consumer2 */ + "t1", 1, "t1", 3, "t2", 1, "t2", 4, "t2", 6, "t3", 0, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + /* One consumer on a rack with no partitions */ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 3, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 2, "t2", 3, "t3", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 4, "t2", 5, "t2", 6, "t3", 1, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 2); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testRackAwareAssignmentWithUniformPartitions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + char *topics[] = {"t1", "t2", "t3"}; + int partitions[] = {5, 5, 5}; + int partitions_mismatch[] = {10, 5, 3}; + rd_kafka_group_member_t members[3]; + size_t i = 0; + int replication_factor = 0; + int subscriptions_count[] = {3, 3, 3}; + char **subscriptions[] = {topics, topics, topics}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + /* Verify combinations where rack-aware logic is not used. */ + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, + /* consumer1*/ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t3", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 2, "t3", 3, NULL, + /* consumer3 */ + "t1", 4, "t2", 4, "t3", 4, NULL); + + /* Verify that co-partitioning is prioritized over rack-alignment for + * topics with equal subscriptions */ + for (replication_factor = 1; replication_factor <= 3; + replication_factor++) { + rd_kafka_metadata_t *metadata = NULL; + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), + replication_factor, replication_factor < 3 ? 3 : 2, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1*/ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t3", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 2, "t3", 3, NULL, + /* consumer3 */ + "t1", 4, "t2", 4, "t3", 4, NULL); + verifyNumPartitionsWithRackMismatch( + metadata, members, RD_ARRAYSIZE(members), + partitions_mismatch[replication_factor - 1]); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + } + + RD_UT_PASS(); +} + +static int ut_testRackAwareAssignmentWithUniformPartitionsNonEqualSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_metadata_t *metadata = NULL; + char *topics[] = {"t1", "t2", "t3"}; + int partitions[] = {5, 5, 5}; + rd_kafka_group_member_t members[3]; + size_t i = 0; + int subscriptions_count[] = {3, 3, 2}; + char *subscription13[] = {"t1", "t3"}; + char **subscriptions[] = {topics, topics, subscription13}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + /* Verify combinations where rack-aware logic is not used. */ + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, + /* consumer1*/ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 2, "t3", 0, "t3", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 3, "t2", 4, "t3", 2, "t3", 3, NULL, + /* consumer3 */ + "t1", 4, "t3", 4, NULL); + + /* Verify that co-partitioning is prioritized over rack-alignment for + * topics with equal subscriptions */ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 4, "t3", 0, "t3", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 2, "t3", 3, NULL, + /* consumer3 */ + "t1", 4, "t3", 4, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 9); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 2, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 2, "t2", 0, "t2", 1, "t2", 3, "t3", 2, NULL, + /* consumer2 */ + "t1", 0, "t1", 3, "t2", 2, "t2", 4, "t3", 0, "t3", 3, NULL, + /* consumer3 */ + "t1", 1, "t1", 4, "t3", 1, "t3", 4, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + /* One consumer on a rack with no partitions */ + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 3, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t2", 2, "t3", 0, "t3", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 3, "t2", 4, "t3", 2, "t3", 3, NULL, + /* consumer3 */ + "t1", 4, "t3", 4, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 2); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testRackAwareAssignmentWithCoPartitioning0( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_metadata_t *metadata = NULL; + char *topics[] = {"t1", "t2", "t3", "t4"}; + int partitions[] = {6, 6, 2, 2}; + rd_kafka_group_member_t members[4]; + size_t i = 0; + int subscriptions_count[] = {2, 2, 2, 2}; + char *subscription12[] = {"t1", "t2"}; + char *subscription34[] = {"t3", "t4"}; + char **subscriptions[] = {subscription12, subscription12, + subscription34, subscription34}; + int racks[] = {0, 1, 1, 0}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + setupRackAwareAssignment(rk, rkas, members, RD_ARRAYSIZE(members), 3, 2, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t1", 2, "t2", 0, "t2", 1, "t2", 2, NULL, + /* consumer2 */ + "t1", 3, "t1", 4, "t1", 5, "t2", 3, "t2", 4, "t2", 5, NULL, + /* consumer3 */ + "t3", 0, "t4", 0, NULL, + /* consumer4 */ + "t3", 1, "t4", 1, NULL); + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 2, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t1", 2, "t2", 0, "t2", 1, "t2", 2, NULL, + /* consumer2 */ + "t1", 3, "t1", 4, "t1", 5, "t2", 3, "t2", 4, "t2", 5, NULL, + /* consumer3 */ + "t3", 0, "t4", 0, NULL, + /* consumer4 */ + "t3", 1, "t4", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 2, "t1", 4, "t2", 0, "t2", 2, "t2", 4, NULL, + /* consumer2 */ + "t1", 1, "t1", 3, "t1", 5, "t2", 1, "t2", 3, "t2", 5, NULL, + /* consumer3 */ + "t3", 1, "t4", 1, NULL, + /* consumer4 */ + "t3", 0, "t4", 0, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testRackAwareAssignmentWithCoPartitioning1( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_metadata_t *metadata = NULL; + char *topics[] = {"t1", "t2", "t3", "t4"}; + int partitions[] = {6, 6, 2, 2}; + rd_kafka_group_member_t members[4]; + size_t i = 0; + int subscriptions_count[] = {4, 4, 4, 4}; + char **subscriptions[] = {topics, topics, topics, topics}; + int racks[] = {0, 1, 1, 0}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + setupRackAwareAssignment(rk, rkas, members, RD_ARRAYSIZE(members), 3, 2, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t4", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 1, "t4", 1, NULL, + /* consumer3 */ + "t1", 4, "t2", 4, NULL, + /* consumer4 */ + "t1", 5, "t2", 5, NULL); + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 2, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t4", 0, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 1, "t4", 1, NULL, + /* consumer3 */ + "t1", 4, "t2", 4, NULL, + /* consumer4 */ + "t1", 5, "t2", 5, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 2, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 2, "t2", 0, "t2", 2, "t3", 0, "t4", 0, NULL, + /* consumer2 */ + "t1", 1, "t1", 3, "t2", 1, "t2", 3, "t3", 1, "t4", 1, NULL, + /* consumer3 */ + "t1", 5, "t2", 5, NULL, + /* consumer4 */ + "t1", 4, "t2", 4, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, racks, + &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t3", 0, "t4", 0, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 1, "t4", 1, NULL, + /* consumer3 */ + "t1", 2, "t2", 2, NULL, + /* consumer4 */ + "t1", 5, "t2", 5, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 6); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testCoPartitionedAssignmentWithSameSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_metadata_t *metadata = NULL; + char *topics[] = {"t1", "t2", "t3", "t4", "t5", "t6"}; + int partitions[] = {6, 6, 2, 2, 4, 4}; + rd_kafka_group_member_t members[3]; + size_t i = 0; + int subscriptions_count[] = {6, 6, 6}; + char **subscriptions[] = {topics, topics, topics}; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + setupRackAwareAssignment(rk, rkas, members, RD_ARRAYSIZE(members), 3, 0, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t4", 0, "t5", 0, "t5", + 1, "t6", 0, "t6", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 1, "t4", 1, "t5", 2, "t6", + 2, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, "t2", 4, "t2", 5, "t5", 3, "t6", 3, NULL); + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 3, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 1, "t2", 0, "t2", 1, "t3", 0, "t4", 0, "t5", 0, "t5", + 1, "t6", 0, "t6", 1, NULL, + /* consumer2 */ + "t1", 2, "t1", 3, "t2", 2, "t2", 3, "t3", 1, "t4", 1, "t5", 2, "t6", + 2, NULL, + /* consumer3 */ + "t1", 4, "t1", 5, "t2", 4, "t2", 5, "t5", 3, "t6", 3, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int rd_kafka_range_assignor_unittest(void) { + rd_kafka_conf_t *conf; + rd_kafka_t *rk; + int fails = 0; + char errstr[256]; + rd_kafka_assignor_t *rkas; + size_t i; + + conf = rd_kafka_conf_new(); + if (rd_kafka_conf_set(conf, "group.id", "test", errstr, + sizeof(errstr)) || + rd_kafka_conf_set(conf, "partition.assignment.strategy", "range", + errstr, sizeof(errstr))) + RD_UT_FAIL("range assignor conf failed: %s", errstr); + + rd_kafka_conf_set(conf, "debug", rd_getenv("TEST_DEBUG", NULL), NULL, + 0); + + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + RD_UT_ASSERT(rk, "range assignor client instantiation failed: %s", + errstr); + rkas = rd_kafka_assignor_find(rk, "range"); + RD_UT_ASSERT(rkas, "range assignor not found"); + + for (i = 0; i < RD_ARRAY_SIZE(ALL_RACKS) - 1; i++) { + char c = 'a' + i; + ALL_RACKS[i] = rd_kafkap_str_new(&c, 1); + } + ALL_RACKS[i] = NULL; + + static int (*tests[])( + rd_kafka_t *, const rd_kafka_assignor_t *, + rd_kafka_assignor_ut_rack_config_t parametrization) = { + ut_testOneConsumerNoTopic, + ut_testOneConsumerNonexistentTopic, + ut_testOneConsumerOneTopic, + ut_testOnlyAssignsPartitionsFromSubscribedTopics, + ut_testOneConsumerMultipleTopics, + ut_testTwoConsumersOneTopicOnePartition, + ut_testTwoConsumersOneTopicTwoPartitions, + ut_testMultipleConsumersMixedTopicSubscriptions, + ut_testTwoConsumersTwoTopicsSixPartitions, + ut_testRackAwareAssignmentWithUniformSubscription, + ut_testRackAwareAssignmentWithNonEqualSubscription, + ut_testRackAwareAssignmentWithUniformPartitions, + ut_testRackAwareAssignmentWithUniformPartitionsNonEqualSubscription, + ut_testRackAwareAssignmentWithCoPartitioning0, + ut_testRackAwareAssignmentWithCoPartitioning1, + ut_testCoPartitionedAssignmentWithSameSubscription, + NULL, + }; + + for (i = 0; tests[i]; i++) { + rd_ts_t ts = rd_clock(); + int r = 0; + rd_kafka_assignor_ut_rack_config_t j; + + for (j = RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK; + j != RD_KAFKA_RANGE_ASSIGNOR_UT_CONFIG_CNT; j++) { + RD_UT_SAY("[ Test #%" PRIusz ", RackConfig = %d ]", i, + j); + r += tests[i](rk, rkas, j); + } + RD_UT_SAY("[ Test #%" PRIusz " ran for %.3fms ]", i, + (double)(rd_clock() - ts) / 1000.0); + + RD_UT_ASSERT(!r, "^ failed"); + + fails += r; + } + + for (i = 0; i < RD_ARRAY_SIZE(ALL_RACKS) - 1; i++) { + rd_kafkap_str_destroy(ALL_RACKS[i]); + } + + rd_kafka_destroy(rk); + + return fails; +} + + /** * @brief Initialzie and add range assignor. @@ -133,6 +1741,7 @@ rd_kafka_resp_err_t rd_kafka_range_assignor_init(rd_kafka_t *rk) { return rd_kafka_assignor_add( rk, "consumer", "range", RD_KAFKA_REBALANCE_PROTOCOL_EAGER, rd_kafka_range_assignor_assign_cb, - rd_kafka_assignor_get_metadata_with_empty_userdata, NULL, NULL, - NULL, NULL); + rd_kafka_assignor_get_metadata_with_empty_userdata, + NULL /* on_assignment_cb */, NULL /* destroy_state_cb */, + rd_kafka_range_assignor_unittest, NULL); } diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 64abded1ec..d7fa17dca5 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -1765,7 +1765,8 @@ void rd_kafka_JoinGroupRequest(rd_kafka_broker_t *rkb, rd_kafka_buf_write_kstr(rkbuf, rkas->rkas_protocol_name); member_metadata = rkas->rkas_get_metadata_cb( rkas, rk->rk_cgrp->rkcg_assignor_state, topics, - rk->rk_cgrp->rkcg_group_assignment); + rk->rk_cgrp->rkcg_group_assignment, + rk->rk_conf.client_rack); rd_kafka_buf_write_kbytes(rkbuf, member_metadata); rd_kafkap_bytes_destroy(member_metadata); } @@ -2087,14 +2088,11 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_kafka_buf_t *rkbuf, rd_kafka_buf_t *request, void *opaque) { - rd_kafka_op_t *rko = opaque; /* Possibly NULL */ - struct rd_kafka_metadata *md = NULL; - rd_kafka_topic_authorized_operations_pair_t - *topic_authorized_operations = NULL; - int32_t cluster_authorized_operations; - char *cluster_id = NULL; - int controller_id; - const rd_list_t *topics = request->rkbuf_u.Metadata.topics; + rd_kafka_op_t *rko = opaque; /* Possibly NULL */ + rd_kafka_metadata_internal_t *mdi = NULL; + // rd_kafka_topic_authorized_operations_pair_t + // *topic_authorized_operations = NULL; + const rd_list_t *topics = request->rkbuf_u.Metadata.topics; int actions; rd_kafka_assert(NULL, err == RD_KAFKA_RESP_ERR__DESTROY || @@ -2121,26 +2119,21 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_list_cnt(topics), request->rkbuf_u.Metadata.reason); - err = rd_kafka_parse_Metadata( - rkb, request, rkbuf, &md, &topic_authorized_operations, - &cluster_authorized_operations, NULL, &cluster_id, &controller_id); + err = rd_kafka_parse_Metadata(rkb, request, rkbuf, &mdi, NULL); if (err) goto err; if (rko && rko->rko_replyq.q) { /* Reply to metadata requester, passing on the metadata. * Reuse requesting rko for the reply. */ - rko->rko_err = err; - rko->rko_u.metadata.md = md; - rko->rko_u.metadata.cluster_authorized_operations = - cluster_authorized_operations; - rko->rko_u.metadata.topic_authorized_operations = - topic_authorized_operations; + rko->rko_err = err; + rko->rko_u.metadata.md = &mdi->metadata; + rko->rko_u.metadata.mdi = mdi; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); rko = NULL; } else { - if (md) - rd_free(md); + if (mdi) + rd_free(mdi); } goto done; @@ -2166,10 +2159,9 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, rd_kafka_actions2str(actions)); /* Respond back to caller on non-retriable errors */ if (rko && rko->rko_replyq.q) { - rko->rko_err = err; - rko->rko_u.metadata.md = NULL; - rko->rko_u.metadata.topic_authorized_operations = NULL; - rko->rko_u.metadata.cluster_authorized_operations = -1; + rko->rko_err = err; + rko->rko_u.metadata.md = NULL; + rko->rko_u.metadata.mdi = NULL; rd_kafka_replyq_enq(&rko->rko_replyq, rko, 0); rko = NULL; } @@ -2204,6 +2196,8 @@ static void rd_kafka_handle_Metadata(rd_kafka_t *rk, * @param include_topic_authorized_operations - request for topic authorized * operations. * @param cgrp_update - Update cgrp in parse_Metadata (see comment there). + * @param force_racks - Force partition to rack mapping computation in + * parse_Metadata (see comment there). * @param rko - (optional) rko with replyq for handling response. * Specifying an rko forces a metadata request even if * there is already a matching one in-transit. @@ -2225,6 +2219,7 @@ rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_bool_t include_cluster_authorized_operations, rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, + rd_bool_t force_racks, rd_kafka_op_t *rko, rd_kafka_resp_cb_t *resp_cb, int force, @@ -2248,6 +2243,7 @@ rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rkbuf->rkbuf_u.Metadata.reason = rd_strdup(reason); rkbuf->rkbuf_u.Metadata.cgrp_update = cgrp_update; + rkbuf->rkbuf_u.Metadata.force_racks = force_racks; /* TopicArrayCnt */ of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 5398a8f121..56d1683eef 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -263,6 +263,7 @@ rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, rd_bool_t include_cluster_authorized_operations, rd_bool_t include_topic_authorized_operations, rd_bool_t cgrp_update, + rd_bool_t force_racks, rd_kafka_op_t *rko, rd_kafka_resp_cb_t *resp_cb, int force, diff --git a/src/rdkafka_sticky_assignor.c b/src/rdkafka_sticky_assignor.c index 8e76ddb14e..8d2bc27982 100644 --- a/src/rdkafka_sticky_assignor.c +++ b/src/rdkafka_sticky_assignor.c @@ -160,6 +160,9 @@ typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *, typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *, rd_list_t *) map_toppar_list_t; +typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *, + rd_kafka_metadata_partition_internal_t *) map_toppar_mdpi_t; + typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *, ConsumerGenerationPair_t *) map_toppar_cgpair_t; @@ -173,6 +176,7 @@ typedef RD_MAP_TYPE(const ConsumerPair_t *, typedef RD_MAP_TYPE(const char *, map_cpair_toppar_list_t *) map_str_map_cpair_toppar_list_t; +typedef RD_MAP_TYPE(const char *, const char *) map_str_str_t; /** Glue type helpers */ @@ -193,6 +197,121 @@ static void map_cpair_toppar_list_t_free(void *ptr) { } +/** @struct Convenience struct for storing consumer/rack and toppar/rack + * mappings. */ +typedef struct { + /** A map of member_id -> rack_id pairs. */ + map_str_str_t member_id_to_rack_id; + /* A map of topic partition to rd_kafka_metadata_partition_internal_t */ + map_toppar_mdpi_t toppar_to_mdpi; +} rd_kafka_rack_info_t; + +/** + * @brief Initialize a rd_kafka_rack_info_t. + * + * @param topics + * @param topic_cnt + * @param mdi + * + * This struct is for convenience/easy grouping, and as a consequence, we avoid + * copying values. Thus, it is intended to be used within the lifetime of this + * function's arguments. + * + * @return rd_kafka_rack_info_t* + */ +static rd_kafka_rack_info_t * +rd_kafka_rack_info_new(rd_kafka_assignor_topic_t **topics, + size_t topic_cnt, + const rd_kafka_metadata_internal_t *mdi) { + int i; + size_t t; + rd_kafka_group_member_t *rkgm; + rd_kafka_rack_info_t *rkri = rd_calloc(1, sizeof(rd_kafka_rack_info_t)); + + if (!rd_kafka_use_rack_aware_assignment(topics, topic_cnt, mdi)) { + /* Free everything immediately, we aren't using rack aware + assignment, this struct is not applicable. */ + rd_free(rkri); + return NULL; + } + + rkri->member_id_to_rack_id = (map_str_str_t)RD_MAP_INITIALIZER( + 0, rd_map_str_cmp, rd_map_str_hash, + NULL /* refs members.rkgm_member_id */, + NULL /* refs members.rkgm_rack_id */); + rkri->toppar_to_mdpi = (map_toppar_mdpi_t)RD_MAP_INITIALIZER( + 0, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash, + rd_kafka_topic_partition_destroy_free, NULL); + + for (t = 0; t < topic_cnt; t++) { + RD_LIST_FOREACH(rkgm, &topics[t]->members, i) { + RD_MAP_SET(&rkri->member_id_to_rack_id, + rkgm->rkgm_member_id->str, + rkgm->rkgm_rack_id->str); + } + + for (i = 0; i < topics[t]->metadata->partition_cnt; i++) { + rd_kafka_topic_partition_t *rkpart = + rd_kafka_topic_partition_new( + topics[t]->metadata->topic, i); + RD_MAP_SET( + &rkri->toppar_to_mdpi, rkpart, + &topics[t]->metadata_internal->partitions[i]); + } + } + + return rkri; +} + +/* Destroy a rd_kafka_rack_info_t. */ +static void rd_kafka_rack_info_destroy(rd_kafka_rack_info_t *rkri) { + if (!rkri) + return; + + RD_MAP_DESTROY(&rkri->member_id_to_rack_id); + RD_MAP_DESTROY(&rkri->toppar_to_mdpi); + + rd_free(rkri); +} + + +/* Convenience function to bsearch inside the racks of a + * rd_kafka_metadata_partition_internal_t. */ +static char *rd_kafka_partition_internal_find_rack( + rd_kafka_metadata_partition_internal_t *mdpi, + const char *rack) { + char **partition_racks = mdpi->racks; + size_t cnt = mdpi->racks_cnt; + + void *res = + bsearch(&rack, partition_racks, cnt, sizeof(char *), rd_strcmp3); + + if (res) + return *(char **)res; + return NULL; +} + + +/* Computes whether there is a rack mismatch between the rack of the consumer + * and the topic partition/any of its replicas. */ +static rd_bool_t +rd_kafka_racks_mismatch(rd_kafka_rack_info_t *rkri, + const char *consumer, + const rd_kafka_topic_partition_t *topic_partition) { + const char *consumer_rack; + rd_kafka_metadata_partition_internal_t *mdpi; + + if (rkri == NULL) /* Not using rack aware assignment */ + return rd_false; + + consumer_rack = RD_MAP_GET(&rkri->member_id_to_rack_id, consumer); + + mdpi = RD_MAP_GET(&rkri->toppar_to_mdpi, topic_partition); + + return consumer_rack != NULL && + (mdpi == NULL || + !rd_kafka_partition_internal_find_rack(mdpi, consumer_rack)); +} /** * @struct Provides current state of partition movements between consumers @@ -399,13 +518,15 @@ static int sort_by_map_elem_val_toppar_list_cnt(const void *_a, * * The assignment should improve the overall balance of the partition * assignments to consumers. + * @returns true if partition was assigned, false otherwise. */ -static void -assignPartition(const rd_kafka_topic_partition_t *partition, - rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/, - map_str_toppar_list_t *currentAssignment, - map_str_toppar_list_t *consumer2AllPotentialPartitions, - map_toppar_str_t *currentPartitionConsumer) { +static rd_bool_t +maybeAssignPartition(const rd_kafka_topic_partition_t *partition, + rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/, + map_str_toppar_list_t *currentAssignment, + map_str_toppar_list_t *consumer2AllPotentialPartitions, + map_toppar_str_t *currentPartitionConsumer, + rd_kafka_rack_info_t *rkri) { const rd_map_elem_t *elem; int i; @@ -418,6 +539,9 @@ assignPartition(const rd_kafka_topic_partition_t *partition, if (!rd_kafka_topic_partition_list_find( partitions, partition->topic, partition->partition)) continue; + if (rkri != NULL && + rd_kafka_racks_mismatch(rkri, consumer, partition)) + continue; rd_kafka_topic_partition_list_add( RD_MAP_GET(currentAssignment, consumer), partition->topic, @@ -431,8 +555,9 @@ assignPartition(const rd_kafka_topic_partition_t *partition, * This is an O(N) operation since it is a single shuffle. */ rd_list_sort(sortedCurrentSubscriptions, sort_by_map_elem_val_toppar_list_cnt); - return; + return rd_true; } + return rd_false; } /** @@ -639,14 +764,6 @@ isBalanced(rd_kafka_t *rk, ->value) ->cnt; - /* Mapping from partitions to the consumer assigned to them */ - // FIXME: don't create prior to min/max check below */ - map_toppar_str_t allPartitions = RD_MAP_INITIALIZER( - RD_MAP_CNT(partition2AllPotentialConsumers), - rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash, - NULL /* references currentAssignment */, - NULL /* references currentAssignment */); - /* Iterators */ const rd_kafka_topic_partition_list_t *partitions; const char *consumer; @@ -661,10 +778,16 @@ isBalanced(rd_kafka_t *rk, "minimum %d and maximum %d partitions assigned " "to each consumer", minimum, maximum); - RD_MAP_DESTROY(&allPartitions); return rd_true; } + /* Mapping from partitions to the consumer assigned to them */ + map_toppar_str_t allPartitions = RD_MAP_INITIALIZER( + RD_MAP_CNT(partition2AllPotentialConsumers), + rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash, + NULL /* references currentAssignment */, + NULL /* references currentAssignment */); + /* Create a mapping from partitions to the consumer assigned to them */ RD_MAP_FOREACH(consumer, partitions, currentAssignment) { @@ -764,7 +887,8 @@ performReassignments(rd_kafka_t *rk, rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/, map_str_toppar_list_t *consumer2AllPotentialPartitions, map_toppar_list_t *partition2AllPotentialConsumers, - map_toppar_str_t *currentPartitionConsumer) { + map_toppar_str_t *currentPartitionConsumer, + rd_kafka_rack_info_t *rkri) { rd_bool_t reassignmentPerformed = rd_false; rd_bool_t modified, saveIsBalanced = rd_false; int iterations = 0; @@ -796,6 +920,9 @@ performReassignments(rd_kafka_t *rk, const ConsumerGenerationPair_t *prevcgp; const rd_kafka_topic_partition_list_t *currAssignment; int j; + rd_bool_t found_rack; + const char *consumer_rack = NULL; + rd_kafka_metadata_partition_internal_t *mdpi = NULL; /* FIXME: Is this a local error/bug? If so, assert */ if (rd_list_cnt(consumers) <= 1) @@ -832,7 +959,59 @@ performReassignments(rd_kafka_t *rk, } /* Check if a better-suited consumer exists for the - * partition; if so, reassign it. */ + * partition; if so, reassign it. Use consumer within + * rack if possible. */ + if (rkri) { + consumer_rack = RD_MAP_GET( + &rkri->member_id_to_rack_id, consumer); + mdpi = RD_MAP_GET(&rkri->toppar_to_mdpi, + partition); + } + found_rack = rd_false; + + if (consumer_rack != NULL && mdpi != NULL && + mdpi->racks_cnt > 0 && + rd_kafka_partition_internal_find_rack( + mdpi, consumer_rack)) { + RD_LIST_FOREACH(otherConsumer, consumers, j) { + /* No need for rkri == NULL check, that + * is guaranteed if we're inside this if + * block. */ + const char *other_consumer_rack = + RD_MAP_GET( + &rkri->member_id_to_rack_id, + otherConsumer); + + if (other_consumer_rack == NULL || + !rd_kafka_partition_internal_find_rack( + mdpi, other_consumer_rack)) + continue; + + if (currAssignment->cnt <= + RD_MAP_GET(currentAssignment, + otherConsumer) + ->cnt + + 1) + continue; + + reassignPartition( + rk, partitionMovements, partition, + currentAssignment, + sortedCurrentSubscriptions, + currentPartitionConsumer, + consumer2AllPotentialPartitions); + + reassignmentPerformed = rd_true; + modified = rd_true; + found_rack = rd_true; + break; + } + } + + if (found_rack) { + continue; + } + RD_LIST_FOREACH(otherConsumer, consumers, j) { if (consumer == otherConsumer) continue; @@ -911,7 +1090,43 @@ static int getBalanceScore(map_str_toppar_list_t *assignment) { return score; } +static void maybeAssign(rd_kafka_topic_partition_list_t *unassignedPartitions, + map_toppar_list_t *partition2AllPotentialConsumers, + rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/, + map_str_toppar_list_t *currentAssignment, + map_str_toppar_list_t *consumer2AllPotentialPartitions, + map_toppar_str_t *currentPartitionConsumer, + rd_bool_t removeAssigned, + rd_kafka_rack_info_t *rkri) { + int i; + const rd_kafka_topic_partition_t *partition; + + for (i = 0; i < unassignedPartitions->cnt; i++) { + partition = &unassignedPartitions->elems[i]; + rd_bool_t assigned; + + /* Skip if there is no potential consumer for the partition. + * FIXME: How could this be? */ + if (rd_list_empty(RD_MAP_GET(partition2AllPotentialConsumers, + partition))) { + rd_dassert(!*"sticky assignor bug"); + continue; + } + assigned = maybeAssignPartition( + partition, sortedCurrentSubscriptions, currentAssignment, + consumer2AllPotentialPartitions, currentPartitionConsumer, + rkri); + if (assigned && removeAssigned) { + rd_kafka_topic_partition_list_del_by_idx( + unassignedPartitions, i); + i--; /* Since the current element was + * removed we need the next for + * loop iteration to stay at the + * same index. */ + } + } +} /** * @brief Balance the current assignment using the data structures @@ -926,7 +1141,8 @@ static void balance(rd_kafka_t *rk, map_str_toppar_list_t *consumer2AllPotentialPartitions, map_toppar_list_t *partition2AllPotentialConsumers, map_toppar_str_t *currentPartitionConsumer, - rd_bool_t revocationRequired) { + rd_bool_t revocationRequired, + rd_kafka_rack_info_t *rkri) { /* If the consumer with most assignments (thus the last element * in the ascendingly ordered sortedCurrentSubscriptions list) has @@ -964,23 +1180,34 @@ static void balance(rd_kafka_t *rk, const void *ignore; const rd_map_elem_t *elem; int i; - - /* Assign all unassigned partitions */ - for (i = 0; i < unassignedPartitions->cnt; i++) { - partition = &unassignedPartitions->elems[i]; - - /* Skip if there is no potential consumer for the partition. - * FIXME: How could this be? */ - if (rd_list_empty(RD_MAP_GET(partition2AllPotentialConsumers, - partition))) { - rd_dassert(!*"sticky assignor bug"); - continue; - } - - assignPartition( - partition, sortedCurrentSubscriptions, currentAssignment, - consumer2AllPotentialPartitions, currentPartitionConsumer); + rd_kafka_topic_partition_list_t *leftoverUnassignedPartitions; + rd_bool_t leftoverUnassignedPartitions_allocated = rd_false; + + leftoverUnassignedPartitions = + unassignedPartitions; /* copy on write. */ + + if (rkri != NULL && RD_MAP_CNT(&rkri->member_id_to_rack_id) != 0) { + leftoverUnassignedPartitions_allocated = rd_true; + /* Since maybeAssign is called twice, we keep track of those + * partitions which the first call has taken care of already, + * but we don't want to modify the original + * unassignedPartitions. */ + leftoverUnassignedPartitions = + rd_kafka_topic_partition_list_copy(unassignedPartitions); + maybeAssign(leftoverUnassignedPartitions, + partition2AllPotentialConsumers, + sortedCurrentSubscriptions, currentAssignment, + consumer2AllPotentialPartitions, + currentPartitionConsumer, rd_true, rkri); } + maybeAssign(leftoverUnassignedPartitions, + partition2AllPotentialConsumers, sortedCurrentSubscriptions, + currentAssignment, consumer2AllPotentialPartitions, + currentPartitionConsumer, rd_false, NULL); + + if (leftoverUnassignedPartitions_allocated) + rd_kafka_topic_partition_list_destroy( + leftoverUnassignedPartitions); /* Narrow down the reassignment scope to only those partitions that can @@ -1050,17 +1277,18 @@ static void balance(rd_kafka_t *rk, * changes, first try to balance by only moving newly added partitions. */ if (!revocationRequired && unassignedPartitions->cnt > 0) - performReassignments( - rk, partitionMovements, unassignedPartitions, - currentAssignment, prevAssignment, - sortedCurrentSubscriptions, consumer2AllPotentialPartitions, - partition2AllPotentialConsumers, currentPartitionConsumer); + performReassignments(rk, partitionMovements, + unassignedPartitions, currentAssignment, + prevAssignment, sortedCurrentSubscriptions, + consumer2AllPotentialPartitions, + partition2AllPotentialConsumers, + currentPartitionConsumer, rkri); reassignmentPerformed = performReassignments( rk, partitionMovements, sortedPartitions, currentAssignment, prevAssignment, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, - currentPartitionConsumer); + currentPartitionConsumer, rkri); /* If we are not preserving existing assignments and we have made * changes to the current assignment make sure we are getting a more @@ -1180,24 +1408,6 @@ static void prepopulateCurrentAssignments( &sortedPartitionConsumersByGeneration, partition, rd_list_new(10, ConsumerGenerationPair_destroy)); - if (consumer->rkgm_generation != -1 && - rd_list_find( - consumers, &consumer->rkgm_generation, - ConsumerGenerationPair_cmp_generation)) { - rd_kafka_log( - rk, LOG_WARNING, "STICKY", - "Sticky assignor: " - "%s [%" PRId32 - "] is assigned to " - "multiple consumers with same " - "generation %d: " - "skipping member %.*s", - partition->topic, partition->partition, - consumer->rkgm_generation, - RD_KAFKAP_STR_PR(consumer->rkgm_member_id)); - continue; - } - rd_list_add(consumers, ConsumerGenerationPair_new( consumer->rkgm_member_id->str, @@ -1215,24 +1425,55 @@ static void prepopulateCurrentAssignments( RD_MAP_FOREACH(partition, consumers, &sortedPartitionConsumersByGeneration) { /* current and previous are the last two consumers - * of each partition. */ - ConsumerGenerationPair_t *current, *previous; + * of each partition, and found is used to check for duplicate + * consumers of same generation. */ + ConsumerGenerationPair_t *current, *previous, *found; rd_kafka_topic_partition_list_t *partitions; /* Sort the per-partition consumers list by generation */ rd_list_sort(consumers, ConsumerGenerationPair_cmp_generation); + /* In case a partition is claimed by multiple consumers with the + * same generation, invalidate it for all such consumers, and + * log an error for this situation. */ + if ((found = rd_list_find_duplicate( + consumers, ConsumerGenerationPair_cmp_generation))) { + const char *consumer1, *consumer2; + int idx = rd_list_index( + consumers, found, + ConsumerGenerationPair_cmp_generation); + consumer1 = ((ConsumerGenerationPair_t *)rd_list_elem( + consumers, idx)) + ->consumer; + consumer2 = ((ConsumerGenerationPair_t *)rd_list_elem( + consumers, idx + 1)) + ->consumer; + + RD_MAP_DELETE(currentPartitionConsumer, partition); + + rd_kafka_log( + rk, LOG_ERR, "STICKY", + "Sticky assignor: Found multiple consumers %s and " + "%s claiming the same topic partition %s:%d in the " + "same generation %d, this will be invalidated and " + "removed from their previous assignment.", + consumer1, consumer2, partition->topic, + partition->partition, found->generation); + continue; + } + /* Add current (highest generation) consumer * to currentAssignment. */ - current = rd_list_elem(consumers, 0); + current = rd_list_last(consumers); partitions = RD_MAP_GET(currentAssignment, current->consumer); rd_kafka_topic_partition_list_add(partitions, partition->topic, partition->partition); /* Add previous (next highest generation) consumer, if any, * to prevAssignment. */ - previous = rd_list_elem(consumers, 1); - if (previous) + if (rd_list_cnt(consumers) >= 2 && + (previous = + rd_list_elem(consumers, rd_list_cnt(consumers) - 2))) RD_MAP_SET( prevAssignment, rd_kafka_topic_partition_copy(partition), @@ -1590,6 +1831,11 @@ rd_kafka_sticky_assignor_assign_cb(rd_kafka_t *rk, void *opaque) { /* FIXME: Let the cgrp pass the actual eligible partition count */ size_t partition_cnt = member_cnt * 10; /* FIXME */ + const rd_kafka_metadata_internal_t *mdi = + rd_kafka_metadata_get_internal(metadata); + + rd_kafka_rack_info_t *rkri = + rd_kafka_rack_info_new(eligible_topics, eligible_topic_cnt, mdi); /* Map of subscriptions. This is \p member turned into a map. */ map_str_toppar_list_t subscriptions = @@ -1680,6 +1926,10 @@ rd_kafka_sticky_assignor_assign_cb(rd_kafka_t *rk, unassignedPartitions = rd_kafka_topic_partition_list_copy(sortedPartitions); + if (rkri) + rd_kafka_dbg(rk, CGRP, "STICKY", + "Sticky assignor: using rack aware assignment."); + RD_MAP_FOREACH(consumer, partitions, ¤tAssignment) { if (!RD_MAP_GET(&subscriptions, consumer)) { /* If a consumer that existed before @@ -1726,13 +1976,16 @@ rd_kafka_sticky_assignor_assign_cb(rd_kafka_t *rk, RD_MAP_GET(&subscriptions, consumer), partition->topic, - RD_KAFKA_PARTITION_UA)) { + RD_KAFKA_PARTITION_UA) || + rd_kafka_racks_mismatch( + rkri, consumer, partition)) { /* If this partition cannot remain * assigned to its current consumer * because the consumer is no longer - * subscribed to its topic, remove it - * from the currentAssignment of the - * consumer. */ + * subscribed to its topic, or racks + * don't match for rack-aware + * assignment, remove it from the + * currentAssignment of the consumer. */ remove_part = rd_true; revocationRequired = rd_true; } else { @@ -1785,7 +2038,7 @@ rd_kafka_sticky_assignor_assign_cb(rd_kafka_t *rk, sortedPartitions, unassignedPartitions, &sortedCurrentSubscriptions, &consumer2AllPotentialPartitions, &partition2AllPotentialConsumers, ¤tPartitionConsumer, - revocationRequired); + revocationRequired, rkri); /* Transfer currentAssignment (now updated) to each member's * assignment. */ @@ -1798,6 +2051,7 @@ rd_kafka_sticky_assignor_assign_cb(rd_kafka_t *rk, rd_kafka_topic_partition_list_destroy(unassignedPartitions); rd_kafka_topic_partition_list_destroy(sortedPartitions); + rd_kafka_rack_info_destroy(rkri); RD_MAP_DESTROY(¤tPartitionConsumer); RD_MAP_DESTROY(&consumer2AllPotentialPartitions); @@ -1837,7 +2091,8 @@ static rd_kafkap_bytes_t *rd_kafka_sticky_assignor_get_metadata( const rd_kafka_assignor_t *rkas, void *assignor_state, const rd_list_t *topics, - const rd_kafka_topic_partition_list_t *owned_partitions) { + const rd_kafka_topic_partition_list_t *owned_partitions, + const rd_kafkap_str_t *rack_id) { rd_kafka_sticky_assignor_state_t *state; rd_kafka_buf_t *rkbuf; rd_kafkap_bytes_t *metadata; @@ -1855,9 +2110,11 @@ static rd_kafkap_bytes_t *rd_kafka_sticky_assignor_get_metadata( * If there is no previous assignment, UserData is NULL. */ + if (!assignor_state) { return rd_kafka_consumer_protocol_member_metadata_new( - topics, NULL, 0, owned_partitions); + topics, NULL, 0, owned_partitions, -1 /* generation */, + rack_id); } state = (rd_kafka_sticky_assignor_state_t *)assignor_state; @@ -1880,7 +2137,8 @@ static rd_kafkap_bytes_t *rd_kafka_sticky_assignor_get_metadata( rd_kafka_buf_destroy(rkbuf); metadata = rd_kafka_consumer_protocol_member_metadata_new( - topics, kbytes->data, kbytes->len, owned_partitions); + topics, kbytes->data, kbytes->len, owned_partitions, + state->generation_id, rack_id); rd_kafkap_bytes_destroy(kbytes); @@ -1913,296 +2171,97 @@ static void rd_kafka_sticky_assignor_state_destroy(void *assignor_state) { * */ - - -/** - * @brief Set a member's owned partitions based on its assignment. - * - * For use between assignor_run(). This is mimicing a consumer receiving - * its new assignment and including it in the next rebalance as its - * owned-partitions. - */ -static void ut_set_owned(rd_kafka_group_member_t *rkgm) { - if (rkgm->rkgm_owned) - rd_kafka_topic_partition_list_destroy(rkgm->rkgm_owned); - - rkgm->rkgm_owned = - rd_kafka_topic_partition_list_copy(rkgm->rkgm_assignment); -} - - -/** - * @brief Verify assignment validity and balance. - * - * @remark Also updates the members owned partitions to the assignment. - */ - -static int verifyValidityAndBalance0(const char *func, - int line, - rd_kafka_group_member_t *members, - size_t member_cnt, - const rd_kafka_metadata_t *metadata) { - int fails = 0; - int i; - rd_bool_t verbose = rd_false; /* Enable for troubleshooting */ - - RD_UT_SAY("%s:%d: verifying assignment for %d member(s):", func, line, - (int)member_cnt); - - for (i = 0; i < (int)member_cnt; i++) { - const char *consumer = members[i].rkgm_member_id->str; - const rd_kafka_topic_partition_list_t *partitions = - members[i].rkgm_assignment; - int p, j; - - if (verbose) - RD_UT_SAY( - "%s:%d: " - "consumer \"%s\", %d subscribed topic(s), " - "%d assigned partition(s):", - func, line, consumer, - members[i].rkgm_subscription->cnt, partitions->cnt); - - for (p = 0; p < partitions->cnt; p++) { - const rd_kafka_topic_partition_t *partition = - &partitions->elems[p]; - - if (verbose) - RD_UT_SAY("%s:%d: %s [%" PRId32 "]", func, - line, partition->topic, - partition->partition); - - if (!rd_kafka_topic_partition_list_find( - members[i].rkgm_subscription, partition->topic, - RD_KAFKA_PARTITION_UA)) { - RD_UT_WARN("%s [%" PRId32 - "] is assigned to " - "%s but it is not subscribed to " - "that topic", - partition->topic, - partition->partition, consumer); - fails++; - } - } - - /* Update the member's owned partitions to match - * the assignment. */ - ut_set_owned(&members[i]); - - if (i == (int)member_cnt - 1) - continue; - - for (j = i + 1; j < (int)member_cnt; j++) { - const char *otherConsumer = - members[j].rkgm_member_id->str; - const rd_kafka_topic_partition_list_t *otherPartitions = - members[j].rkgm_assignment; - rd_bool_t balanced = - abs(partitions->cnt - otherPartitions->cnt) <= 1; - - for (p = 0; p < partitions->cnt; p++) { - const rd_kafka_topic_partition_t *partition = - &partitions->elems[p]; - - if (rd_kafka_topic_partition_list_find( - otherPartitions, partition->topic, - partition->partition)) { - RD_UT_WARN( - "Consumer %s and %s are both " - "assigned %s [%" PRId32 "]", - consumer, otherConsumer, - partition->topic, - partition->partition); - fails++; - } - - - /* If assignment is imbalanced and this topic - * is also subscribed by the other consumer - * it means the assignment strategy failed to - * properly balance the partitions. */ - if (!balanced && - rd_kafka_topic_partition_list_find_topic( - otherPartitions, partition->topic)) { - RD_UT_WARN( - "Some %s partition(s) can be " - "moved from " - "%s (%d partition(s)) to " - "%s (%d partition(s)) to " - "achieve a better balance", - partition->topic, consumer, - partitions->cnt, otherConsumer, - otherPartitions->cnt); - fails++; - } - } - } - } - - RD_UT_ASSERT(!fails, "%s:%d: See %d previous errors", func, line, - fails); - - return 0; -} - - -#define verifyValidityAndBalance(members, member_cnt, metadata) \ - do { \ - if (verifyValidityAndBalance0(__FUNCTION__, __LINE__, members, \ - member_cnt, metadata)) \ - return 1; \ - } while (0) - - -/** - * @brief Checks that all assigned partitions are fully balanced. - * - * Only works for symmetrical subscriptions. - */ -static int isFullyBalanced0(const char *function, - int line, - const rd_kafka_group_member_t *members, - size_t member_cnt) { - int min_assignment = INT_MAX; - int max_assignment = -1; - size_t i; - - for (i = 0; i < member_cnt; i++) { - int size = members[i].rkgm_assignment->cnt; - if (size < min_assignment) - min_assignment = size; - if (size > max_assignment) - max_assignment = size; - } - - RD_UT_ASSERT(max_assignment - min_assignment <= 1, - "%s:%d: Assignment not balanced: min %d, max %d", function, - line, min_assignment, max_assignment); - - return 0; +/* All possible racks used in tests, as well as several common rack configs used + * by consumers */ +static rd_kafkap_str_t + *ALL_RACKS[7]; /* initialized before starting the unit tests. */ +static int RACKS_INITIAL[] = {0, 1, 2}; +static int RACKS_NULL[] = {6, 6, 6}; +static int RACKS_FINAL[] = {4, 5, 6}; +static int RACKS_ONE_NULL[] = {6, 4, 5}; + +/* Helper to get consumer rack based on the index of the consumer. */ +static rd_kafkap_str_t * +ut_get_consumer_rack(int idx, + rd_kafka_assignor_ut_rack_config_t parametrization) { + const int cycle_size = + (parametrization == RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK + ? RD_ARRAYSIZE(ALL_RACKS) + : 3); + return (ALL_RACKS[idx % cycle_size]); } -#define isFullyBalanced(members, member_cnt) \ - do { \ - if (isFullyBalanced0(__FUNCTION__, __LINE__, members, \ - member_cnt)) \ - return 1; \ - } while (0) - - +/* Helper to populate a member's owned partitions (accepted as variadic), and + * generation. */ static void -ut_print_toppar_list(const rd_kafka_topic_partition_list_t *partitions) { - int i; - - for (i = 0; i < partitions->cnt; i++) - RD_UT_SAY(" %s [%" PRId32 "]", partitions->elems[i].topic, - partitions->elems[i].partition); -} - - - -/** - * @brief Verify that member's assignment matches the expected partitions. - * - * The va-list is a NULL-terminated list of (const char *topic, int partition) - * tuples. - * - * @returns 0 on success, else raises a unittest error and returns 1. - */ -static int verifyAssignment0(const char *function, - int line, - rd_kafka_group_member_t *rkgm, - ...) { +ut_populate_member_owned_partitions_generation(rd_kafka_group_member_t *rkgm, + int generation, + size_t partition_cnt, + ...) { va_list ap; - int cnt = 0; - const char *topic; - int fails = 0; + size_t i; - va_start(ap, rkgm); - while ((topic = va_arg(ap, const char *))) { - int partition = va_arg(ap, int); - cnt++; + if (rkgm->rkgm_owned) + rd_kafka_topic_partition_list_destroy(rkgm->rkgm_owned); + rkgm->rkgm_owned = rd_kafka_topic_partition_list_new(partition_cnt); - if (!rd_kafka_topic_partition_list_find(rkgm->rkgm_assignment, - topic, partition)) { - RD_UT_WARN( - "%s:%d: Expected %s [%d] not found in %s's " - "assignment (%d partition(s))", - function, line, topic, partition, - rkgm->rkgm_member_id->str, - rkgm->rkgm_assignment->cnt); - fails++; - } + va_start(ap, partition_cnt); + for (i = 0; i < partition_cnt; i++) { + char *topic = va_arg(ap, char *); + int partition = va_arg(ap, int); + rd_kafka_topic_partition_list_add(rkgm->rkgm_owned, topic, + partition); } va_end(ap); - if (cnt != rkgm->rkgm_assignment->cnt) { - RD_UT_WARN( - "%s:%d: " - "Expected %d assigned partition(s) for %s, not %d", - function, line, cnt, rkgm->rkgm_member_id->str, - rkgm->rkgm_assignment->cnt); - fails++; - } - - if (fails) - ut_print_toppar_list(rkgm->rkgm_assignment); - - RD_UT_ASSERT(!fails, "%s:%d: See previous errors", function, line); - - return 0; + rkgm->rkgm_generation = generation; } -#define verifyAssignment(rkgm, ...) \ - do { \ - if (verifyAssignment0(__FUNCTION__, __LINE__, rkgm, \ - __VA_ARGS__)) \ - return 1; \ - } while (0) - - - -/** - * @brief Initialize group member struct for testing. - * - * va-args is a NULL-terminated list of (const char *) topics. - * - * Use rd_kafka_group_member_clear() to free fields. - */ -static void -ut_init_member(rd_kafka_group_member_t *rkgm, const char *member_id, ...) { +/* Helper to create topic partition list from a variadic list of topic, + * partition pairs. */ +static rd_kafka_topic_partition_list_t ** +ut_create_topic_partition_lists(size_t list_cnt, ...) { va_list ap; - const char *topic; - - memset(rkgm, 0, sizeof(*rkgm)); - - rkgm->rkgm_member_id = rd_kafkap_str_new(member_id, -1); - rkgm->rkgm_group_instance_id = rd_kafkap_str_new(member_id, -1); - rd_list_init(&rkgm->rkgm_eligible, 0, NULL); - - rkgm->rkgm_subscription = rd_kafka_topic_partition_list_new(4); - - va_start(ap, member_id); - while ((topic = va_arg(ap, const char *))) - rd_kafka_topic_partition_list_add(rkgm->rkgm_subscription, - topic, RD_KAFKA_PARTITION_UA); + size_t i; + rd_kafka_topic_partition_list_t **lists = + rd_calloc(list_cnt, sizeof(rd_kafka_topic_partition_list_t *)); + + va_start(ap, list_cnt); + for (i = 0; i < list_cnt; i++) { + const char *topic; + lists[i] = rd_kafka_topic_partition_list_new(0); + while ((topic = va_arg(ap, const char *))) { + int partition = va_arg(ap, int); + rd_kafka_topic_partition_list_add(lists[i], topic, + partition); + } + } va_end(ap); - rkgm->rkgm_assignment = - rd_kafka_topic_partition_list_new(rkgm->rkgm_subscription->size); + return lists; } - - -static int ut_testOneConsumerNoTopic(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int +ut_testOneConsumerNoTopic(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = rd_kafka_metadata_new_topic_mock(NULL, 0); - ut_init_member(&members[0], "consumer1", "topic1", NULL); + if (parametrization == RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { + RD_UT_PASS(); + } + + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 0); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2214,21 +2273,32 @@ static int ut_testOneConsumerNoTopic(rd_kafka_t *rk, isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testOneConsumerNonexistentTopic(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testOneConsumerNonexistentTopic( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 0); - ut_init_member(&members[0], "consumer1", "topic1", NULL); + if (parametrization == RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK) { + RD_UT_PASS(); + } + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 0); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2240,22 +2310,29 @@ static int ut_testOneConsumerNonexistentTopic(rd_kafka_t *rk, isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testOneConsumerOneTopic(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int +ut_testOneConsumerOneTopic(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 3); - ut_init_member(&members[0], "consumer1", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2272,7 +2349,7 @@ static int ut_testOneConsumerOneTopic(rd_kafka_t *rk, isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -2280,16 +2357,20 @@ static int ut_testOneConsumerOneTopic(rd_kafka_t *rk, static int ut_testOnlyAssignsPartitionsFromSubscribedTopics( rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { - + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 3, "topic2", 3); - ut_init_member(&members[0], "consumer1", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2303,22 +2384,28 @@ static int ut_testOnlyAssignsPartitionsFromSubscribedTopics( isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testOneConsumerMultipleTopics(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testOneConsumerMultipleTopics( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 1, "topic2", 2); - ut_init_member(&members[0], "consumer1", "topic1", "topic2", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 1, "topic2", 2); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2332,22 +2419,30 @@ static int ut_testOneConsumerMultipleTopics(rd_kafka_t *rk, isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testTwoConsumersOneTopicOnePartition(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testTwoConsumersOneTopicOnePartition( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[2]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 1); - ut_init_member(&members[0], "consumer1", "topic1", NULL); - ut_init_member(&members[1], "consumer2", "topic1", NULL); + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 1); + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2362,23 +2457,31 @@ ut_testTwoConsumersOneTopicOnePartition(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testTwoConsumersOneTopicTwoPartitions(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testTwoConsumersOneTopicTwoPartitions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[2]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 2); - ut_init_member(&members[0], "consumer1", "topic1", NULL); - ut_init_member(&members[1], "consumer2", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 2); + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2393,7 +2496,7 @@ ut_testTwoConsumersOneTopicTwoPartitions(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -2401,18 +2504,27 @@ ut_testTwoConsumersOneTopicTwoPartitions(rd_kafka_t *rk, static int ut_testMultipleConsumersMixedTopicSubscriptions( rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[3]; - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 3, "topic2", 2); - ut_init_member(&members[0], "consumer1", "topic1", NULL); - ut_init_member(&members[1], "consumer2", "topic1", "topic2", NULL); - ut_init_member(&members[2], "consumer3", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 2); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2429,24 +2541,31 @@ static int ut_testMultipleConsumersMixedTopicSubscriptions( rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); rd_kafka_group_member_clear(&members[2]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testTwoConsumersTwoTopicsSixPartitions(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testTwoConsumersTwoTopicsSixPartitions( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[2]; - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 3, "topic2", 3); - ut_init_member(&members[0], "consumer1", "topic1", "topic2", NULL); - ut_init_member(&members[1], "consumer2", "topic1", "topic2", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2463,21 +2582,29 @@ ut_testTwoConsumersTwoTopicsSixPartitions(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testAddRemoveConsumerOneTopic(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testAddRemoveConsumerOneTopic( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[2]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 3); - ut_init_member(&members[0], "consumer1", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, 1, errstr, sizeof(errstr)); @@ -2490,7 +2617,9 @@ static int ut_testAddRemoveConsumerOneTopic(rd_kafka_t *rk, isFullyBalanced(members, 1); /* Add consumer2 */ - ut_init_member(&members[1], "consumer2", "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2519,7 +2648,7 @@ static int ut_testAddRemoveConsumerOneTopic(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -2545,25 +2674,35 @@ static int ut_testAddRemoveConsumerOneTopic(rd_kafka_t *rk, * - consumer3: topic1-1, topic5-0 * - consumer4: topic4-0, topic5-1 */ -static int -ut_testPoorRoundRobinAssignmentScenario(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testPoorRoundRobinAssignmentScenario( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[4]; - metadata = rd_kafka_metadata_new_topic_mockv( - 5, "topic1", 2, "topic2", 1, "topic3", 2, "topic4", 1, "topic5", 2); - - ut_init_member(&members[0], "consumer1", "topic1", "topic2", "topic3", - "topic4", "topic5", NULL); - ut_init_member(&members[1], "consumer2", "topic1", "topic3", "topic5", - NULL); - ut_init_member(&members[2], "consumer3", "topic1", "topic3", "topic5", - NULL); - ut_init_member(&members[3], "consumer4", "topic1", "topic2", "topic3", - "topic4", "topic5", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 5, "topic1", 2, "topic2", 1, "topic3", 2, + "topic4", 1, "topic5", 2); + + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", + "topic3", "topic4", "topic5", NULL); + ut_initMemberConditionalRack( + &members[1], "consumer2", ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic3", "topic5", NULL); + ut_initMemberConditionalRack( + &members[2], "consumer3", ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", "topic3", "topic5", NULL); + ut_initMemberConditionalRack(&members[3], "consumer4", + ut_get_consumer_rack(3, parametrization), + parametrization, "topic1", "topic2", + "topic3", "topic4", "topic5", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2582,23 +2721,32 @@ ut_testPoorRoundRobinAssignmentScenario(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[1]); rd_kafka_group_member_clear(&members[2]); rd_kafka_group_member_clear(&members[3]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testAddRemoveTopicTwoConsumers(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testAddRemoveTopicTwoConsumers( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[2]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 3); - ut_init_member(&members[0], "consumer1", "topic1", "topic2", NULL); - ut_init_member(&members[1], "consumer2", "topic1", "topic2", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2615,9 +2763,11 @@ static int ut_testAddRemoveTopicTwoConsumers(rd_kafka_t *rk, * Add topic2 */ RD_UT_SAY("Adding topic2"); - rd_kafka_metadata_destroy(metadata); - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 3, "topic2", 3); + ut_destroy_metadata(metadata); + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2638,8 +2788,11 @@ static int ut_testAddRemoveTopicTwoConsumers(rd_kafka_t *rk, * Remove topic1 */ RD_UT_SAY("Removing topic1"); - rd_kafka_metadata_destroy(metadata); - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic2", 3); + ut_destroy_metadata(metadata); + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic2", 3); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -2655,15 +2808,16 @@ static int ut_testAddRemoveTopicTwoConsumers(rd_kafka_t *rk, rd_kafka_group_member_clear(&members[0]); rd_kafka_group_member_clear(&members[1]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testReassignmentAfterOneConsumerLeaves(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testReassignmentAfterOneConsumerLeaves( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2680,8 +2834,9 @@ ut_testReassignmentAfterOneConsumerLeaves(rd_kafka_t *rk, mt[i].partition_cnt = i + 1; } - metadata = rd_kafka_metadata_new_topic_mock(mt, topic_cnt); - + ut_initMetadataConditionalRack0(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), + parametrization, mt, topic_cnt); for (i = 1; i <= member_cnt; i++) { char name[20]; @@ -2695,7 +2850,12 @@ ut_testReassignmentAfterOneConsumerLeaves(rd_kafka_t *rk, subscription, topic, RD_KAFKA_PARTITION_UA); } rd_snprintf(name, sizeof(name), "consumer%d", i); - ut_init_member(&members[i - 1], name, NULL); + + ut_initMemberConditionalRack( + &members[i - 1], name, + ut_get_consumer_rack(i, parametrization), parametrization, + NULL); + rd_kafka_topic_partition_list_destroy( members[i - 1].rkgm_subscription); members[i - 1].rkgm_subscription = subscription; @@ -2725,15 +2885,16 @@ ut_testReassignmentAfterOneConsumerLeaves(rd_kafka_t *rk, for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testReassignmentAfterOneConsumerAdded(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testReassignmentAfterOneConsumerAdded( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2741,7 +2902,9 @@ ut_testReassignmentAfterOneConsumerAdded(rd_kafka_t *rk, int member_cnt = RD_ARRAYSIZE(members); int i; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 20); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 20); for (i = 1; i <= member_cnt; i++) { char name[20]; @@ -2750,7 +2913,10 @@ ut_testReassignmentAfterOneConsumerAdded(rd_kafka_t *rk, rd_kafka_topic_partition_list_add(subscription, "topic1", RD_KAFKA_PARTITION_UA); rd_snprintf(name, sizeof(name), "consumer%d", i); - ut_init_member(&members[i - 1], name, NULL); + ut_initMemberConditionalRack( + &members[i - 1], name, + ut_get_consumer_rack(i, parametrization), parametrization, + NULL); rd_kafka_topic_partition_list_destroy( members[i - 1].rkgm_subscription); members[i - 1].rkgm_subscription = subscription; @@ -2778,14 +2944,16 @@ ut_testReassignmentAfterOneConsumerAdded(rd_kafka_t *rk, for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testSameSubscriptions(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int +ut_testSameSubscriptions(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2806,12 +2974,17 @@ static int ut_testSameSubscriptions(rd_kafka_t *rk, RD_KAFKA_PARTITION_UA); } - metadata = rd_kafka_metadata_new_topic_mock(mt, topic_cnt); + ut_initMetadataConditionalRack0(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), + parametrization, mt, topic_cnt); for (i = 1; i <= member_cnt; i++) { char name[16]; rd_snprintf(name, sizeof(name), "consumer%d", i); - ut_init_member(&members[i - 1], name, NULL); + ut_initMemberConditionalRack( + &members[i - 1], name, + ut_get_consumer_rack(i, parametrization), parametrization, + NULL); rd_kafka_topic_partition_list_destroy( members[i - 1].rkgm_subscription); members[i - 1].rkgm_subscription = @@ -2840,7 +3013,7 @@ static int ut_testSameSubscriptions(rd_kafka_t *rk, for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); rd_kafka_topic_partition_list_destroy(subscription); RD_UT_PASS(); @@ -2849,8 +3022,8 @@ static int ut_testSameSubscriptions(rd_kafka_t *rk, static int ut_testLargeAssignmentWithMultipleConsumersLeaving( rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { - + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2867,7 +3040,9 @@ static int ut_testLargeAssignmentWithMultipleConsumersLeaving( mt[i].partition_cnt = i + 1; } - metadata = rd_kafka_metadata_new_topic_mock(mt, topic_cnt); + ut_initMetadataConditionalRack0(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), + parametrization, mt, topic_cnt); for (i = 0; i < member_cnt; i++) { /* Java tests use a random set, this is more deterministic. */ @@ -2884,7 +3059,10 @@ static int ut_testLargeAssignmentWithMultipleConsumersLeaving( RD_KAFKA_PARTITION_UA); rd_snprintf(name, sizeof(name), "consumer%d", i + 1); - ut_init_member(&members[i], name, NULL); + ut_initMemberConditionalRack( + &members[i], name, ut_get_consumer_rack(i, parametrization), + parametrization, NULL); + rd_kafka_topic_partition_list_destroy( members[i].rkgm_subscription); members[i].rkgm_subscription = subscription; @@ -2915,14 +3093,16 @@ static int ut_testLargeAssignmentWithMultipleConsumersLeaving( for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testNewSubscription(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int +ut_testNewSubscription(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2930,15 +3110,19 @@ static int ut_testNewSubscription(rd_kafka_t *rk, int member_cnt = RD_ARRAYSIZE(members); int i; - metadata = rd_kafka_metadata_new_topic_mockv( - 5, "topic1", 1, "topic2", 2, "topic3", 3, "topic4", 4, "topic5", 5); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 5, "topic1", 1, "topic2", 2, "topic3", 3, + "topic4", 4, "topic5", 5); for (i = 0; i < member_cnt; i++) { char name[16]; int j; rd_snprintf(name, sizeof(name), "consumer%d", i); - ut_init_member(&members[i], name, NULL); + ut_initMemberConditionalRack( + &members[i], name, ut_get_consumer_rack(i, parametrization), + parametrization, NULL); rd_kafka_topic_partition_list_destroy( members[i].rkgm_subscription); @@ -2977,14 +3161,16 @@ static int ut_testNewSubscription(rd_kafka_t *rk, for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int ut_testMoveExistingAssignments(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testMoveExistingAssignments( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -2994,12 +3180,22 @@ static int ut_testMoveExistingAssignments(rd_kafka_t *rk, int i; int fails = 0; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 3); - - ut_init_member(&members[0], "consumer1", "topic1", NULL); - ut_init_member(&members[1], "consumer2", "topic1", NULL); - ut_init_member(&members[2], "consumer3", "topic1", NULL); - ut_init_member(&members[3], "consumer4", "topic1", NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[3], "consumer4", + ut_get_consumer_rack(3, parametrization), + parametrization, "topic1", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, member_cnt, errstr, sizeof(errstr)); @@ -3060,14 +3256,75 @@ static int ut_testMoveExistingAssignments(rd_kafka_t *rk, if (assignments[i]) rd_kafka_topic_partition_list_destroy(assignments[i]); } - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } +/* The original version of this test diverged from the Java implementaion in + * what it was testing. It's not certain whether it was by mistake, or by + * design, but the new version matches the Java implementation, and the old one + * is retained as well, since it provides extra coverage. + */ +static int ut_testMoveExistingAssignments_j( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + int member_cnt = RD_ARRAYSIZE(members); + rd_kafka_topic_partition_list_t *assignments[4] = RD_ZERO_INIT; + int i; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 6, "topic1", 1, "topic2", 1, "topic3", 1, + "topic4", 1, "topic5", 1, "topic6", 1); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], 1 /* generation */, 1, "topic1", 0); + + ut_initMemberConditionalRack( + &members[1], "consumer2", ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", "topic3", "topic4", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], 1 /* generation */, 2, "topic2", 0, "topic3", 0); + + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic2", "topic3", + "topic4", "topic5", "topic6", NULL); + ut_populate_member_owned_partitions_generation( + &members[2], 1 /* generation */, 3, "topic4", 0, "topic5", 0, + "topic6", 0); -static int ut_testStickiness(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, member_cnt, metadata); + + for (i = 0; i < member_cnt; i++) { + rd_kafka_group_member_clear(&members[i]); + if (assignments[i]) + rd_kafka_topic_partition_list_destroy(assignments[i]); + } + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int +ut_testStickiness(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -3075,18 +3332,22 @@ static int ut_testStickiness(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { int member_cnt = RD_ARRAYSIZE(members); int i; - metadata = rd_kafka_metadata_new_topic_mockv( - 6, "topic1", 1, "topic2", 1, "topic3", 1, "topic4", 1, "topic5", 1, - "topic6", 1); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 6, "topic1", 1, "topic2", 1, "topic3", 1, + "topic4", 1, "topic5", 1, "topic6", 1); - ut_init_member(&members[0], "consumer1", "topic1", "topic2", NULL); + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); rd_kafka_topic_partition_list_destroy(members[0].rkgm_assignment); members[0].rkgm_assignment = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(members[0].rkgm_assignment, "topic1", 0); - ut_init_member(&members[1], "consumer2", "topic1", "topic2", "topic3", - "topic4", NULL); + ut_initMemberConditionalRack( + &members[1], "consumer2", ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", "topic3", "topic4", NULL); rd_kafka_topic_partition_list_destroy(members[1].rkgm_assignment); members[1].rkgm_assignment = rd_kafka_topic_partition_list_new(2); rd_kafka_topic_partition_list_add(members[1].rkgm_assignment, "topic2", @@ -3094,8 +3355,9 @@ static int ut_testStickiness(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { rd_kafka_topic_partition_list_add(members[1].rkgm_assignment, "topic3", 0); - ut_init_member(&members[2], "consumer3", "topic4", "topic5", "topic6", - NULL); + ut_initMemberConditionalRack( + &members[2], "consumer3", ut_get_consumer_rack(1, parametrization), + parametrization, "topic4", "topic5", "topic6", NULL); rd_kafka_topic_partition_list_destroy(members[2].rkgm_assignment); members[2].rkgm_assignment = rd_kafka_topic_partition_list_new(3); rd_kafka_topic_partition_list_add(members[2].rkgm_assignment, "topic4", @@ -3112,10 +3374,113 @@ static int ut_testStickiness(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); - for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +/* The original version of this test diverged from the Java implementaion in + * what it was testing. It's not certain whether it was by mistake, or by + * design, but the new version matches the Java implementation, and the old one + * is retained as well, for extra coverage. + */ +static int +ut_testStickiness_j(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[4]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + rd_kafka_topic_partition_list_t *assignments[4] = RD_ZERO_INIT; + int fails = 0; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[3], "consumer4", + ut_get_consumer_rack(3, parametrization), + parametrization, "topic1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, member_cnt, metadata); + + for (i = 0; i < member_cnt; i++) { + if (members[i].rkgm_assignment->cnt > 1) { + RD_UT_WARN("%s assigned %d partitions, expected <= 1", + members[i].rkgm_member_id->str, + members[i].rkgm_assignment->cnt); + fails++; + } else if (members[i].rkgm_assignment->cnt == 1) { + assignments[i] = rd_kafka_topic_partition_list_copy( + members[i].rkgm_assignment); + } + } + + /* + * Remove potential group leader consumer1, by starting members at + * index 1. + * Owned partitions of the members are already set to the assignment by + * verifyValidityAndBalance above to simulate the fact that the assignor + * has already run once. + */ + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, &members[1], + member_cnt - 1, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(&members[1], member_cnt - 1, metadata); + // FIXME: isSticky() + + for (i = 1; i < member_cnt; i++) { + if (members[i].rkgm_assignment->cnt != 1) { + RD_UT_WARN("%s assigned %d partitions, expected 1", + members[i].rkgm_member_id->str, + members[i].rkgm_assignment->cnt); + fails++; + } else if (assignments[i] && + !rd_kafka_topic_partition_list_find( + assignments[i], + members[i].rkgm_assignment->elems[0].topic, + members[i] + .rkgm_assignment->elems[0] + .partition)) { + RD_UT_WARN( + "Stickiness was not honored for %s, " + "%s [%" PRId32 "] not in previous assignment", + members[i].rkgm_member_id->str, + members[i].rkgm_assignment->elems[0].topic, + members[i].rkgm_assignment->elems[0].partition); + fails++; + } + } + + RD_UT_ASSERT(!fails, "See previous errors"); + + + for (i = 0; i < member_cnt; i++) { + rd_kafka_group_member_clear(&members[i]); + if (assignments[i]) + rd_kafka_topic_partition_list_destroy(assignments[i]); + } + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -3124,7 +3489,10 @@ static int ut_testStickiness(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { /** * @brief Verify stickiness across three rebalances. */ -static int ut_testStickiness2(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { +static int +ut_testStickiness2(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -3132,11 +3500,19 @@ static int ut_testStickiness2(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { int member_cnt = RD_ARRAYSIZE(members); int i; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 6); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 6); - ut_init_member(&members[0], "consumer1", "topic1", NULL); - ut_init_member(&members[1], "consumer2", "topic1", NULL); - ut_init_member(&members[2], "consumer3", "topic1", NULL); + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); /* Just consumer1 */ err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, 1, @@ -3198,24 +3574,28 @@ static int ut_testStickiness2(rd_kafka_t *rk, const rd_kafka_assignor_t *rkas) { for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testAssignmentUpdatedForDeletedTopic(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testAssignmentUpdatedForDeletedTopic( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = - rd_kafka_metadata_new_topic_mockv(2, "topic1", 1, "topic3", 100); - ut_init_member(&members[0], "consumer1", "topic1", "topic2", "topic3", - NULL); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 1, "topic3", 100); + + ut_initMemberConditionalRack( + &members[0], "consumer1", ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", "topic3", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -3230,7 +3610,7 @@ ut_testAssignmentUpdatedForDeletedTopic(rd_kafka_t *rk, members[0].rkgm_assignment->cnt); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -3238,16 +3618,21 @@ ut_testAssignmentUpdatedForDeletedTopic(rd_kafka_t *rk, static int ut_testNoExceptionThrownWhenOnlySubscribedTopicDeleted( rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { - + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; rd_kafka_group_member_t members[1]; - metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 3); + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); - ut_init_member(&members[0], "consumer1", "topic", NULL); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -3260,8 +3645,8 @@ static int ut_testNoExceptionThrownWhenOnlySubscribedTopicDeleted( /* * Remove topic */ - rd_kafka_metadata_destroy(metadata); - metadata = rd_kafka_metadata_new_topic_mock(NULL, 0); + ut_destroy_metadata(metadata); + metadata = rd_kafka_metadata_new_topic_mock(NULL, 0, -1, 0); err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, RD_ARRAYSIZE(members), errstr, @@ -3272,15 +3657,16 @@ static int ut_testNoExceptionThrownWhenOnlySubscribedTopicDeleted( isFullyBalanced(members, RD_ARRAYSIZE(members)); rd_kafka_group_member_clear(&members[0]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } -static int -ut_testConflictingPreviousAssignments(rd_kafka_t *rk, - const rd_kafka_assignor_t *rkas) { +static int ut_testConflictingPreviousAssignments( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { rd_kafka_resp_err_t err; char errstr[512]; rd_kafka_metadata_t *metadata; @@ -3289,6 +3675,8 @@ ut_testConflictingPreviousAssignments(rd_kafka_t *rk, int i; // FIXME: removed from Java test suite, and fails for us, why, why? + // NOTE: rack-awareness changes aren't made to this test because of + // the FIXME above. RD_UT_PASS(); metadata = rd_kafka_metadata_new_topic_mockv(1, "topic1", 2); @@ -3333,7 +3721,7 @@ ut_testConflictingPreviousAssignments(rd_kafka_t *rk, for (i = 0; i < member_cnt; i++) rd_kafka_group_member_clear(&members[i]); - rd_kafka_metadata_destroy(metadata); + ut_destroy_metadata(metadata); RD_UT_PASS(); } @@ -3342,13 +3730,947 @@ ut_testConflictingPreviousAssignments(rd_kafka_t *rk, * from Java since random tests don't provide meaningful test coverage. */ +static int ut_testAllConsumersReachExpectedQuotaAndAreConsideredFilled( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 4); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], 1 /* generation */, 2, "topic1", 0, "topic1", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], 1 /* generation */, 1, "topic1", 2); + + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + verifyAssignment(&members[0], "topic1", 0, "topic1", 1, NULL); + verifyAssignment(&members[1], "topic1", 2, NULL); + verifyAssignment(&members[2], "topic1", 3, NULL); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int ut_testOwnedPartitionsAreInvalidatedForConsumerWithStaleGeneration( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + int current_generation = 10; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], current_generation, 3, "topic1", 0, "topic1", 2, + "topic2", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], current_generation - 1, 3, "topic1", 0, "topic1", 2, + "topic2", 1); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + verifyAssignment(&members[0], "topic1", 0, "topic1", 2, "topic2", 1, + NULL); + verifyAssignment(&members[1], "topic1", 1, "topic2", 0, "topic2", 2, + NULL); + + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int ut_testOwnedPartitionsAreInvalidatedForConsumerWithNoGeneration( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + int current_generation = 10; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], current_generation, 3, "topic1", 0, "topic1", 2, + "topic2", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], -1 /* default generation*/, 3, "topic1", 0, "topic1", + 2, "topic2", 1); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + verifyAssignment(&members[0], "topic1", 0, "topic1", 2, "topic2", 1, + NULL); + verifyAssignment(&members[1], "topic1", 1, "topic2", 0, "topic2", 2, + NULL); + + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +static int +ut_testPartitionsTransferringOwnershipIncludeThePartitionClaimedByMultipleConsumersInSameGeneration( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + // partition topic-0 is owned by multiple consumers + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], 1 /* generation */, 2, "topic1", 0, "topic1", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], 1 /* generation */, 2, "topic1", 0, "topic1", 2); + + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); + + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + verifyAssignment(&members[0], "topic1", 1, NULL); + verifyAssignment(&members[1], "topic1", 2, NULL); + verifyAssignment(&members[2], "topic1", 0, NULL); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +/* In Java, there is a way to check what partition transferred ownership. + * We don't have anything like that for our UTs, so in lieue of that, this + * test is added along with the previous test to make sure that we move the + * right partition. Our solution in case of two consumers owning the same + * partitions with the same generation id was differing from the Java + * implementation earlier. (Check #4252.) */ +static int +ut_testPartitionsTransferringOwnershipIncludeThePartitionClaimedByMultipleConsumersInSameGeneration2( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 1, "topic1", 3); + + // partition topic-0 is owned by multiple consumers + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], 1 /* generation */, 2, "topic1", 0, "topic1", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], 1 /* generation */, 2, "topic1", 1, "topic1", 2); + + ut_initMemberConditionalRack(&members[2], "consumer3", + ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", NULL); + + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + verifyAssignment(&members[0], "topic1", 0, NULL); + verifyAssignment(&members[1], "topic1", 2, NULL); + verifyAssignment(&members[2], "topic1", 1, NULL); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int ut_testEnsurePartitionsAssignedToHighestGeneration( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[3]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + int currentGeneration = 10; + + ut_initMetadataConditionalRack( + &metadata, 3, 3, ALL_RACKS, RD_ARRAYSIZE(ALL_RACKS), + parametrization, 3, "topic1", 3, "topic2", 3, "topic3", 3); + + ut_initMemberConditionalRack( + &members[0], "consumer1", ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], currentGeneration, 3, "topic1", 0, "topic2", 0, + "topic3", 0); + + + ut_initMemberConditionalRack( + &members[1], "consumer2", ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], currentGeneration - 1, 3, "topic1", 1, "topic2", 1, + "topic3", 1); + + + ut_initMemberConditionalRack( + &members[2], "consumer3", ut_get_consumer_rack(2, parametrization), + parametrization, "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[2], currentGeneration - 2, 3, "topic2", 1, "topic3", 0, + "topic3", 2); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + verifyAssignment(&members[0], "topic1", 0, "topic2", 0, "topic3", 0, + NULL); + verifyAssignment(&members[1], "topic1", 1, "topic2", 1, "topic3", 1, + NULL); + verifyAssignment(&members[2], "topic1", 2, "topic2", 2, "topic3", 2, + NULL); + + verifyValidityAndBalance(members, RD_ARRAYSIZE(members), metadata); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int ut_testNoReassignmentOnCurrentMembers( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[4]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + int currentGeneration = 10; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 4, "topic0", 3, "topic1", 3, "topic2", 3, + "topic3", 3); + + ut_initMemberConditionalRack( + &members[0], "consumer1", ut_get_consumer_rack(0, parametrization), + parametrization, "topic0", "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], -1 /* default generation */, 0); + + ut_initMemberConditionalRack( + &members[1], "consumer2", ut_get_consumer_rack(1, parametrization), + parametrization, "topic0", "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], currentGeneration - 1, 3, "topic0", 0, "topic2", 0, + "topic1", 0); + + ut_initMemberConditionalRack( + &members[2], "consumer3", ut_get_consumer_rack(2, parametrization), + parametrization, "topic0", "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[2], currentGeneration - 2, 3, "topic3", 2, "topic2", 2, + "topic1", 1); + + ut_initMemberConditionalRack( + &members[3], "consumer4", ut_get_consumer_rack(3, parametrization), + parametrization, "topic0", "topic1", "topic2", "topic3", NULL); + ut_populate_member_owned_partitions_generation( + &members[3], currentGeneration - 3, 3, "topic3", 1, "topic0", 1, + "topic0", 2); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, member_cnt, metadata); + verifyAssignment(&members[0], "topic1", 2, "topic2", 1, "topic3", 0, + NULL); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + + +static int +ut_testOwnedPartitionsAreInvalidatedForConsumerWithMultipleGeneration( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata; + rd_kafka_group_member_t members[2]; + int member_cnt = RD_ARRAYSIZE(members); + int i; + int currentGeneration = 10; + + ut_initMetadataConditionalRack(&metadata, 3, 3, ALL_RACKS, + RD_ARRAYSIZE(ALL_RACKS), parametrization, + 2, "topic1", 3, "topic2", 3); + + ut_initMemberConditionalRack(&members[0], "consumer1", + ut_get_consumer_rack(0, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[0], currentGeneration, 3, "topic1", 0, "topic2", 1, + "topic1", 1); + + ut_initMemberConditionalRack(&members[1], "consumer2", + ut_get_consumer_rack(1, parametrization), + parametrization, "topic1", "topic2", NULL); + ut_populate_member_owned_partitions_generation( + &members[1], currentGeneration - 2, 3, "topic1", 0, "topic2", 1, + "topic2", 2); + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + verifyValidityAndBalance(members, member_cnt, metadata); + verifyAssignment(&members[0], "topic1", 0, "topic2", 1, "topic1", 1, + NULL); + verifyAssignment(&members[1], "topic1", 2, "topic2", 2, "topic2", 0, + NULL); + + for (i = 0; i < member_cnt; i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + + RD_UT_PASS(); +} + +/* Helper for setting up metadata and members, and running the assignor, and + * verifying validity and balance of the assignment. Does not check the results + * of the assignment on a per member basis.. + */ +static int +setupRackAwareAssignment0(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_group_member_t *members, + size_t member_cnt, + int replication_factor, + int num_broker_racks, + size_t topic_cnt, + char *topics[], + int *partitions, + int *subscriptions_count, + char **subscriptions[], + int *consumer_racks, + rd_kafka_topic_partition_list_t **owned_tp_list, + rd_bool_t initialize_members, + rd_kafka_metadata_t **metadata) { + rd_kafka_resp_err_t err; + char errstr[512]; + rd_kafka_metadata_t *metadata_local = NULL; + + size_t i = 0; + const int num_brokers = num_broker_racks > 0 + ? replication_factor * num_broker_racks + : replication_factor; + if (!metadata) + metadata = &metadata_local; + + /* The member naming for tests is consumerN where N is a single + * character. */ + rd_assert(member_cnt <= 9); + + *metadata = rd_kafka_metadata_new_topic_with_partition_replicas_mock( + replication_factor, num_brokers, topics, partitions, topic_cnt); + ut_populate_internal_broker_metadata( + rd_kafka_metadata_get_internal(*metadata), num_broker_racks, + ALL_RACKS, RD_ARRAYSIZE(ALL_RACKS)); + ut_populate_internal_topic_metadata( + rd_kafka_metadata_get_internal(*metadata)); + + for (i = 0; initialize_members && i < member_cnt; i++) { + char member_id[10]; + snprintf(member_id, 10, "consumer%d", (int)(i + 1)); + ut_init_member_with_rack( + &members[i], member_id, ALL_RACKS[consumer_racks[i]], + subscriptions[i], subscriptions_count[i]); + + if (!owned_tp_list || !owned_tp_list[i]) + continue; + + if (members[i].rkgm_owned) + rd_kafka_topic_partition_list_destroy( + members[i].rkgm_owned); + + members[i].rkgm_owned = + rd_kafka_topic_partition_list_copy(owned_tp_list[i]); + } + + err = rd_kafka_assignor_run(rk->rk_cgrp, rkas, *metadata, members, + member_cnt, errstr, sizeof(errstr)); + RD_UT_ASSERT(!err, "assignor run failed: %s", errstr); + + /* Note that verifyValidityAndBalance also sets rkgm_owned for each + * member to rkgm_assignment, so if the members are used without + * clearing, in another assignor_run, the result should be stable. */ + verifyValidityAndBalance(members, member_cnt, *metadata); + + if (metadata_local) + ut_destroy_metadata(metadata_local); + return 0; +} + +static int +setupRackAwareAssignment(rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_group_member_t *members, + size_t member_cnt, + int replication_factor, + int num_broker_racks, + size_t topic_cnt, + char *topics[], + int *partitions, + int *subscriptions_count, + char **subscriptions[], + int *consumer_racks, + rd_kafka_topic_partition_list_t **owned_tp_list, + rd_bool_t initialize_members) { + return setupRackAwareAssignment0( + rk, rkas, members, member_cnt, replication_factor, num_broker_racks, + topic_cnt, topics, partitions, subscriptions_count, subscriptions, + consumer_racks, owned_tp_list, initialize_members, NULL); +} + +/* Helper for testing cases where rack-aware assignment should not be triggered, + * and assignment should be the same as the pre-rack-aware assignor. Each case + * is run twice, once with owned partitions set to empty, and in the second + * case, with owned partitions set to the result of the previous run, to check + * that the assignment is stable. */ +#define verifyNonRackAwareAssignment(rk, rkas, members, member_cnt, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, ...) \ + do { \ + size_t idx = 0; \ + int init_members = 1; \ + rd_kafka_metadata_t *metadata; \ + \ + /* num_broker_racks = 0, implies that brokers have no \ + * configured racks. */ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment( \ + rk, rkas, members, member_cnt, 3, 0, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_INITIAL, NULL, init_members); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* consumer_racks = RACKS_NULL implies that consumers have no \ + * racks. */ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment( \ + rk, rkas, members, member_cnt, 3, 3, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_NULL, NULL, init_members); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* replication_factor = 3 and num_broker_racks = 3 means that \ + * all partitions are replicated on all racks.*/ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment0( \ + rk, rkas, members, member_cnt, 3, 3, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_INITIAL, NULL, init_members, \ + &metadata); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + verifyNumPartitionsWithRackMismatch( \ + metadata, members, RD_ARRAYSIZE(members), 0); \ + ut_destroy_metadata(metadata); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* replication_factor = 4 and num_broker_racks = 4 means that \ + * all partitions are replicated on all racks. */ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment0( \ + rk, rkas, members, member_cnt, 4, 4, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_INITIAL, NULL, init_members, \ + &metadata); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + verifyNumPartitionsWithRackMismatch( \ + metadata, members, RD_ARRAYSIZE(members), 0); \ + ut_destroy_metadata(metadata); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* There's no overap between broker racks and consumer racks, \ + * since num_broker_racks = 3, they'll be picked from a,b,c \ + * and consumer racks are d,e,f. */ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment( \ + rk, rkas, members, member_cnt, 3, 3, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_FINAL, NULL, init_members); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + /* There's no overap between broker racks and consumer racks, \ + * since num_broker_racks = 3, they'll be picked from a,b,c \ + * and consumer racks are d,e,NULL. */ \ + for (init_members = 1; init_members >= 0; init_members--) { \ + setupRackAwareAssignment( \ + rk, rkas, members, member_cnt, 3, 3, topic_cnt, \ + topics, partitions, subscriptions_count, \ + subscriptions, RACKS_ONE_NULL, NULL, \ + init_members); \ + verifyMultipleAssignment(members, member_cnt, \ + __VA_ARGS__); \ + } \ + for (idx = 0; idx < member_cnt; idx++) \ + rd_kafka_group_member_clear(&members[idx]); \ + } while (0) + + +static int ut_testRackAwareAssignmentWithUniformSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + char *topics[] = {"t1", "t2", "t3"}; + int partitions[] = {6, 7, 2}; + rd_kafka_group_member_t members[3]; + size_t member_cnt = RD_ARRAYSIZE(members); + size_t i = 0; + int subscriptions_count[] = {3, 3, 3}; + char **subscriptions[] = {topics, topics, topics}; + int init_members = 0; + rd_kafka_topic_partition_list_t **owned; + rd_kafka_metadata_t *metadata; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + + /* Verify assignment is rack-aligned for lower replication factor where + * brokers have a subset of partitions */ + for (init_members = 1; init_members >= 0; init_members--) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 1, 3, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + init_members, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + ut_destroy_metadata(metadata); + } + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + + for (init_members = 1; init_members >= 0; init_members--) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 2, 3, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + init_members, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + ut_destroy_metadata(metadata); + } + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + /* One consumer on a rack with no partitions. We allocate with + * misaligned rack to this consumer to maintain balance. */ + for (init_members = 1; init_members >= 0; init_members--) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 3, 2, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + init_members, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 5); + ut_destroy_metadata(metadata); + } + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + /* Verify that rack-awareness is improved if already owned partitions + * are misaligned */ + owned = ut_create_topic_partition_lists( + 3, + /* consumer1 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 3, "t1", 4, NULL, + /* consumer2 */ + "t1", 5, "t2", 0, "t2", 1, "t2", 2, "t2", 3, NULL, + /* consumer3 */ + "t2", 4, "t2", 5, "t2", 6, "t3", 0, "t3", 1, NULL); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, owned, rd_true, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + ut_destroy_metadata(metadata); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + for (i = 0; i < member_cnt; i++) + rd_kafka_topic_partition_list_destroy(owned[i]); + rd_free(owned); + + + /* Verify that stickiness is retained when racks match */ + owned = ut_create_topic_partition_lists( + 3, + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + + /* This test deviates slightly from Java, in that we test with two + * additional replication factors, 1 and 2, which are not tested in + * Java. This is because in Java, there is a way to turn rack aware + * logic on or off for tests. We don't have that, and to test with rack + * aware logic, we need to change something, in this case, the + * replication factor. */ + for (i = 1; i <= 3; i++) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), + i /* replication factor */, 3, RD_ARRAYSIZE(topics), topics, + partitions, subscriptions_count, subscriptions, + RACKS_INITIAL, owned, rd_true, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 0, "t1", 3, "t2", 0, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 1, "t1", 4, "t2", 1, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 2, "t1", 5, "t2", 2, "t2", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + ut_destroy_metadata(metadata); + } + + for (i = 0; i < member_cnt; i++) + rd_kafka_topic_partition_list_destroy(owned[i]); + rd_free(owned); + + RD_UT_PASS(); +} + + +static int ut_testRackAwareAssignmentWithNonEqualSubscription( + rd_kafka_t *rk, + const rd_kafka_assignor_t *rkas, + rd_kafka_assignor_ut_rack_config_t parametrization) { + char *topics[] = {"t1", "t2", "t3"}; + char *topics0[] = {"t1", "t3"}; + int partitions[] = {6, 7, 2}; + rd_kafka_group_member_t members[3]; + size_t member_cnt = RD_ARRAYSIZE(members); + size_t i = 0; + int subscriptions_count[] = {3, 3, 2}; + char **subscriptions[] = {topics, topics, topics0}; + int with_owned = 0; + rd_kafka_topic_partition_list_t **owned; + rd_kafka_metadata_t *metadata; + + if (parametrization != + RD_KAFKA_RANGE_ASSIGNOR_UT_BROKER_AND_CONSUMER_RACK) { + RD_UT_PASS(); + } + + verifyNonRackAwareAssignment( + rk, rkas, members, RD_ARRAYSIZE(members), RD_ARRAYSIZE(topics), + topics, partitions, subscriptions_count, subscriptions, "t1", 5, + "t2", 0, "t2", 2, "t2", 4, "t2", 6, NULL, + /* consumer2 */ + "t1", 3, "t2", 1, "t2", 3, "t2", 5, "t3", 0, NULL, + /* consumer3 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 4, "t3", 1, NULL); + + // Verify assignment is rack-aligned for lower replication factor where + // brokers have a subset of partitions + for (with_owned = 0; with_owned <= 1; with_owned++) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 1, 3, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + !with_owned, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 3, "t2", 0, "t2", 2, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 4, "t2", 1, "t2", 4, "t2", 5, "t3", 0, NULL, + /* consumer3 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 4); + ut_destroy_metadata(metadata); + } + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + + + for (with_owned = 0; with_owned <= 1; with_owned++) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 2, 3, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + !with_owned, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 3, "t2", 0, "t2", 2, "t2", 5, "t2", 6, NULL, + /* consumer2 */ + "t1", 0, "t2", 1, "t2", 3, "t2", 4, "t3", 0, NULL, + /* consumer3 */ + "t1", 1, "t1", 2, "t1", 4, "t1", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 0); + ut_destroy_metadata(metadata); + } + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + /* One consumer on a rack with no partitions. We allocate with + * misaligned rack to this consumer to maintain balance. */ + for (with_owned = 0; with_owned <= 1; with_owned++) { + setupRackAwareAssignment0( + rk, rkas, members, RD_ARRAYSIZE(members), 3, 2, + RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, RACKS_INITIAL, NULL, + !with_owned, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 5, "t2", 0, "t2", 2, "t2", 4, "t2", 6, NULL, + /* consumer2 */ + "t1", 3, "t2", 1, "t2", 3, "t2", 5, "t3", 0, NULL, + /* consumer3 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 4, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 5); + ut_destroy_metadata(metadata); + } + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + + /* Verify that rack-awareness is improved if already owned partitions + * are misaligned. */ + owned = ut_create_topic_partition_lists( + 3, + /* consumer1 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 3, "t1", 4, NULL, + /* consumer2 */ + "t1", 5, "t2", 0, "t2", 1, "t2", 2, "t2", 3, NULL, + /* consumer3 */ + "t2", 4, "t2", 5, "t2", 6, "t3", 0, "t3", 1, NULL); + + setupRackAwareAssignment0(rk, rkas, members, RD_ARRAYSIZE(members), 1, + 3, RD_ARRAYSIZE(topics), topics, partitions, + subscriptions_count, subscriptions, + RACKS_INITIAL, owned, rd_true, &metadata); + verifyMultipleAssignment( + members, RD_ARRAYSIZE(members), + /* consumer1 */ + "t1", 3, "t2", 0, "t2", 2, "t2", 3, "t2", 6, NULL, + /* consumer2 */ + "t1", 4, "t2", 1, "t2", 4, "t2", 5, "t3", 0, NULL, + /* consumer3 */ + "t1", 0, "t1", 1, "t1", 2, "t1", 5, "t3", 1, NULL); + verifyNumPartitionsWithRackMismatch(metadata, members, + RD_ARRAYSIZE(members), 4); + ut_destroy_metadata(metadata); + + for (i = 0; i < RD_ARRAYSIZE(members); i++) + rd_kafka_group_member_clear(&members[i]); + for (i = 0; i < member_cnt; i++) + rd_kafka_topic_partition_list_destroy(owned[i]); + rd_free(owned); + + /* One of the Java tests is skipped here, which tests if the rack-aware + * logic assigns the same partitions as non-rack aware logic. This is + * because we don't have a way to force rack-aware logic like the Java + * assignor. */ + RD_UT_PASS(); +} + static int rd_kafka_sticky_assignor_unittest(void) { rd_kafka_conf_t *conf; rd_kafka_t *rk; int fails = 0; char errstr[256]; rd_kafka_assignor_t *rkas; - static int (*tests[])(rd_kafka_t *, const rd_kafka_assignor_t *) = { + static int (*tests[])( + rd_kafka_t *, const rd_kafka_assignor_t *, + rd_kafka_assignor_ut_rack_config_t parametrization) = { ut_testOneConsumerNoTopic, ut_testOneConsumerNonexistentTopic, ut_testOneConsumerOneTopic, @@ -3367,14 +4689,26 @@ static int rd_kafka_sticky_assignor_unittest(void) { ut_testLargeAssignmentWithMultipleConsumersLeaving, ut_testNewSubscription, ut_testMoveExistingAssignments, + ut_testMoveExistingAssignments_j, ut_testStickiness, + ut_testStickiness_j, ut_testStickiness2, ut_testAssignmentUpdatedForDeletedTopic, ut_testNoExceptionThrownWhenOnlySubscribedTopicDeleted, ut_testConflictingPreviousAssignments, + ut_testAllConsumersReachExpectedQuotaAndAreConsideredFilled, + ut_testOwnedPartitionsAreInvalidatedForConsumerWithStaleGeneration, + ut_testOwnedPartitionsAreInvalidatedForConsumerWithNoGeneration, + ut_testPartitionsTransferringOwnershipIncludeThePartitionClaimedByMultipleConsumersInSameGeneration, + ut_testPartitionsTransferringOwnershipIncludeThePartitionClaimedByMultipleConsumersInSameGeneration2, + ut_testEnsurePartitionsAssignedToHighestGeneration, + ut_testNoReassignmentOnCurrentMembers, + ut_testOwnedPartitionsAreInvalidatedForConsumerWithMultipleGeneration, + ut_testRackAwareAssignmentWithUniformSubscription, + ut_testRackAwareAssignmentWithNonEqualSubscription, NULL, }; - int i; + size_t i; conf = rd_kafka_conf_new(); @@ -3394,13 +4728,25 @@ static int rd_kafka_sticky_assignor_unittest(void) { rkas = rd_kafka_assignor_find(rk, "cooperative-sticky"); RD_UT_ASSERT(rkas, "sticky assignor not found"); + for (i = 0; i < RD_ARRAY_SIZE(ALL_RACKS) - 1; i++) { + char c = 'a' + i; + ALL_RACKS[i] = rd_kafkap_str_new(&c, 1); + } + ALL_RACKS[i] = NULL; + for (i = 0; tests[i]; i++) { rd_ts_t ts = rd_clock(); - int r; - - RD_UT_SAY("[ Test #%d ]", i); - r = tests[i](rk, rkas); - RD_UT_SAY("[ Test #%d ran for %.3fms ]", i, + int r = 0; + rd_kafka_assignor_ut_rack_config_t j; + + RD_UT_SAY("[ Test #%" PRIusz " ]", i); + for (j = RD_KAFKA_RANGE_ASSIGNOR_UT_NO_BROKER_RACK; + j != RD_KAFKA_RANGE_ASSIGNOR_UT_CONFIG_CNT; j++) { + RD_UT_SAY("[ Test #%" PRIusz ", RackConfig = %d ]", i, + j); + r += tests[i](rk, rkas, j); + } + RD_UT_SAY("[ Test #%" PRIusz " ran for %.3fms ]", i, (double)(rd_clock() - ts) / 1000.0); RD_UT_ASSERT(!r, "^ failed"); @@ -3408,6 +4754,10 @@ static int rd_kafka_sticky_assignor_unittest(void) { fails += r; } + for (i = 0; i < RD_ARRAY_SIZE(ALL_RACKS) - 1; i++) { + rd_kafkap_str_destroy(ALL_RACKS[i]); + } + rd_kafka_destroy(rk); return fails; diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 89bfa092df..ba0d18e51f 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -39,6 +39,7 @@ #include "rdsysqueue.h" #include "rdtime.h" #include "rdregex.h" +#include "rdkafka_fetcher.h" #if WITH_ZSTD #include @@ -49,11 +50,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, - const rd_kafka_partition_leader_epoch_t *leader_epochs, - 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_metadata_topic_internal_t *mdit, + rd_ts_t ts_age); /** @@ -478,8 +479,10 @@ 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, NULL, - rkmce->rkmce_ts_insert); + rd_kafka_topic_metadata_update( + rkt, &rkmce->rkmce_mtopic, + &rkmce->rkmce_metadata_internal_topic, + rkmce->rkmce_ts_insert); } if (do_lock) @@ -725,11 +728,16 @@ 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 app pos is real. */ - if (rktp->rktp_app_pos.offset > 0) { - rd_kafka_toppar_set_next_fetch_position( - rktp, rktp->rktp_app_pos); + /* Set offset validation position, + * depending it if should continue with current position or + * with next fetch start position. */ + if (rd_kafka_toppar_fetch_decide_start_from_next_fetch_start( + rktp)) { + rd_kafka_toppar_set_offset_validation_position( + rktp, rktp->rktp_next_fetch_start); + } else { + rd_kafka_toppar_set_offset_validation_position( + rktp, rktp->rktp_offsets.fetch_pos); } rd_kafka_offset_validate(rktp, "epoch updated from metadata"); } @@ -1232,9 +1240,7 @@ 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 mdit Topic internal metadata. * @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. @@ -1242,11 +1248,11 @@ rd_bool_t rd_kafka_topic_set_error(rd_kafka_topic_t *rkt, * * @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, - const rd_kafka_partition_leader_epoch_t *leader_epochs, - 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_metadata_topic_internal_t *mdit, + rd_ts_t ts_age) { rd_kafka_t *rk = rkt->rkt_rk; int upd = 0; int j; @@ -1317,8 +1323,7 @@ static int rd_kafka_topic_metadata_update( 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; + int32_t leader_epoch = mdit->partitions[j].leader_epoch; rd_kafka_dbg(rk, TOPIC | RD_KAFKA_DBG_METADATA, "METADATA", " Topic %s partition %i Leader %" PRId32 @@ -1391,7 +1396,7 @@ static int rd_kafka_topic_metadata_update( 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) { + const rd_kafka_metadata_topic_internal_t *mdit) { rd_kafka_topic_t *rkt; int r; @@ -1402,7 +1407,7 @@ int rd_kafka_topic_metadata_update2( return -1; /* Ignore topics that we dont have locally. */ } - r = rd_kafka_topic_metadata_update(rkt, mdt, leader_epochs, rd_clock()); + r = rd_kafka_topic_metadata_update(rkt, mdt, mdit, rd_clock()); rd_kafka_wrunlock(rkb->rkb_rk); @@ -1749,12 +1754,36 @@ void *rd_kafka_topic_opaque(const rd_kafka_topic_t *app_rkt) { int rd_kafka_topic_info_cmp(const void *_a, const void *_b) { const rd_kafka_topic_info_t *a = _a, *b = _b; - int r; + int r, i; if ((r = strcmp(a->topic, b->topic))) return r; - return RD_CMP(a->partition_cnt, b->partition_cnt); + if ((r = RD_CMP(a->partition_cnt, b->partition_cnt))) + return r; + + if (a->partitions_internal == NULL && b->partitions_internal == NULL) + return 0; + + if (a->partitions_internal == NULL || b->partitions_internal == NULL) + return (a->partitions_internal == NULL) ? 1 : -1; + + /* We're certain partitions_internal exist for a/b and have the same + * count. */ + for (i = 0; i < a->partition_cnt; i++) { + size_t k; + if ((r = RD_CMP(a->partitions_internal[i].racks_cnt, + b->partitions_internal[i].racks_cnt))) + return r; + + for (k = 0; k < a->partitions_internal[i].racks_cnt; k++) { + if ((r = rd_strcmp(a->partitions_internal[i].racks[k], + b->partitions_internal[i].racks[k]))) + return r; + } + } + + return 0; } @@ -1784,7 +1813,77 @@ rd_kafka_topic_info_t *rd_kafka_topic_info_new(const char *topic, ti = rd_malloc(sizeof(*ti) + tlen); ti->topic = (char *)(ti + 1); memcpy((char *)ti->topic, topic, tlen); - ti->partition_cnt = partition_cnt; + ti->partition_cnt = partition_cnt; + ti->partitions_internal = NULL; + + return ti; +} + +/** + * Allocate new topic_info, including rack information. + * \p topic is copied. + */ +rd_kafka_topic_info_t *rd_kafka_topic_info_new_with_rack( + const char *topic, + int partition_cnt, + const rd_kafka_metadata_partition_internal_t *mdpi) { + rd_kafka_topic_info_t *ti; + rd_tmpabuf_t tbuf; + size_t tlen = RD_ROUNDUP(strlen(topic) + 1, 8); + size_t total_racks_size = 0; + int i; + + for (i = 0; i < partition_cnt; i++) { + size_t j; + if (!mdpi[i].racks) + continue; + + for (j = 0; j < mdpi[i].racks_cnt; j++) { + total_racks_size += + RD_ROUNDUP(strlen(mdpi[i].racks[j]) + 1, 8); + } + total_racks_size += + RD_ROUNDUP(sizeof(char *) * mdpi[i].racks_cnt, 8); + } + + if (total_racks_size) /* Only bother allocating this if at least one + rack is there. */ + total_racks_size += + RD_ROUNDUP(sizeof(rd_kafka_metadata_partition_internal_t) * + partition_cnt, + 8); + + rd_tmpabuf_new(&tbuf, sizeof(*ti) + tlen + total_racks_size, + 1 /* assert on fail */); + ti = rd_tmpabuf_alloc(&tbuf, sizeof(*ti)); + ti->topic = rd_tmpabuf_write_str(&tbuf, topic); + ti->partition_cnt = partition_cnt; + ti->partitions_internal = NULL; + + if (total_racks_size) { + ti->partitions_internal = rd_tmpabuf_alloc( + &tbuf, sizeof(*ti->partitions_internal) * partition_cnt); + + for (i = 0; i < partition_cnt; i++) { + size_t j; + ti->partitions_internal[i].id = mdpi[i].id; + ti->partitions_internal[i].racks = NULL; + + if (!mdpi[i].racks) + continue; + + ti->partitions_internal[i].racks_cnt = + mdpi[i].racks_cnt; + ti->partitions_internal[i].racks = rd_tmpabuf_alloc( + &tbuf, sizeof(char *) * mdpi[i].racks_cnt); + + for (j = 0; j < mdpi[i].racks_cnt; j++) { + ti->partitions_internal[i].racks[j] = + rd_tmpabuf_write_str(&tbuf, + mdpi[i].racks[j]); + } + } + } return ti; } @@ -1880,9 +1979,12 @@ void rd_kafka_local_topics_to_list(rd_kafka_t *rk, void rd_ut_kafka_topic_set_topic_exists(rd_kafka_topic_t *rkt, int partition_cnt, int32_t leader_id) { - struct rd_kafka_metadata_topic mdt = {.topic = + rd_kafka_metadata_partition_internal_t *partitions = + rd_calloc(partition_cnt, sizeof(*partitions)); + struct rd_kafka_metadata_topic mdt = {.topic = (char *)rkt->rkt_topic->str, .partition_cnt = partition_cnt}; + rd_kafka_metadata_topic_internal_t mdit = {.partitions = partitions}; int i; mdt.partitions = rd_alloca(sizeof(*mdt.partitions) * partition_cnt); @@ -1894,7 +1996,9 @@ 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, NULL, rd_clock()); + rd_kafka_metadata_cache_topic_update(rkt->rkt_rk, &mdt, &mdit, rd_true, + rd_false, NULL, 0); + rd_kafka_topic_metadata_update(rkt, &mdt, &mdit, rd_clock()); rd_kafka_wrunlock(rkt->rkt_rk); + rd_free(partitions); } diff --git a/src/rdkafka_topic.h b/src/rdkafka_topic.h index 0df289b287..8b7ff894ad 100644 --- a/src/rdkafka_topic.h +++ b/src/rdkafka_topic.h @@ -259,7 +259,7 @@ rd_kafka_topic_get_error(rd_kafka_topic_t *rkt) { 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); + const rd_kafka_metadata_topic_internal_t *mdit); void rd_kafka_topic_scan_all(rd_kafka_t *rk, rd_ts_t now); @@ -267,12 +267,17 @@ void rd_kafka_topic_scan_all(rd_kafka_t *rk, rd_ts_t now); typedef struct rd_kafka_topic_info_s { const char *topic; /**< Allocated along with struct */ int partition_cnt; + rd_kafka_metadata_partition_internal_t *partitions_internal; } rd_kafka_topic_info_t; int rd_kafka_topic_info_topic_cmp(const void *_a, const void *_b); int rd_kafka_topic_info_cmp(const void *_a, const void *_b); rd_kafka_topic_info_t *rd_kafka_topic_info_new(const char *topic, int partition_cnt); +rd_kafka_topic_info_t *rd_kafka_topic_info_new_with_rack( + const char *topic, + int partition_cnt, + const rd_kafka_metadata_partition_internal_t *mdpi); void rd_kafka_topic_info_destroy(rd_kafka_topic_info_t *ti); int rd_kafka_topic_match(rd_kafka_t *rk, diff --git a/src/rdlist.c b/src/rdlist.c index c71e3004ad..82150f99e4 100644 --- a/src/rdlist.c +++ b/src/rdlist.c @@ -148,6 +148,7 @@ void *rd_list_add(rd_list_t *rl, void *elem) { return rl->rl_elems[rl->rl_cnt++]; } + void rd_list_set(rd_list_t *rl, int idx, void *ptr) { if (idx >= rl->rl_size) rd_list_grow(rl, idx + 1); @@ -376,6 +377,34 @@ void *rd_list_find_duplicate(const rd_list_t *rl, return NULL; } +void rd_list_deduplicate(rd_list_t **rl, + int (*cmp)(const void *, const void *)) { + rd_list_t *deduped = rd_list_new(0, (*rl)->rl_free_cb); + void *elem; + void *prev_elem = NULL; + int i; + + if (!((*rl)->rl_flags & RD_LIST_F_SORTED)) + rd_list_sort(*rl, cmp); + + RD_LIST_FOREACH(elem, *rl, i) { + if (prev_elem && cmp(elem, prev_elem) == 0) { + /* Skip this element, and destroy it */ + rd_list_free_cb(*rl, elem); + continue; + } + rd_list_add(deduped, elem); + prev_elem = elem; + } + /* The elements we want destroyed are already destroyed. */ + (*rl)->rl_free_cb = NULL; + rd_list_destroy(*rl); + + /* The parent list was sorted, we can set this without re-sorting. */ + deduped->rl_flags |= RD_LIST_F_SORTED; + *rl = deduped; +} + int rd_list_cmp(const rd_list_t *a, const rd_list_t *b, int (*cmp)(const void *, const void *)) { diff --git a/src/rdlist.h b/src/rdlist.h index db5295f6cf..334b05f916 100644 --- a/src/rdlist.h +++ b/src/rdlist.h @@ -301,6 +301,18 @@ void *rd_list_find_duplicate(const rd_list_t *rl, int (*cmp)(const void *, const void *)); +/** + * @brief Deduplicates a list. + * + * @param rl is a ptrptr since a new list is created and assigned to *rl, for + * efficiency. + * @returns a deduplicated and sorted version of \p *rl. + * @warning the original \p *rl is destroyed. + */ +void rd_list_deduplicate(rd_list_t **rl, + int (*cmp)(const void *, const void *)); + + /** * @brief Compare list \p a to \p b. * diff --git a/src/rdstring.c b/src/rdstring.c index 6a18210c93..2e15cf7e35 100644 --- a/src/rdstring.c +++ b/src/rdstring.c @@ -289,6 +289,21 @@ int rd_strcmp(const char *a, const char *b) { } +/** + * @brief Same as rd_strcmp() but works with rd_list comparator. + */ +int rd_strcmp2(const void *a, const void *b) { + return rd_strcmp((const char *)a, (const char *)b); +} + +/** + * @brief Same as rd_strcmp() but works with bsearch, which requires one more + * indirection. + */ +int rd_strcmp3(const void *a, const void *b) { + return rd_strcmp(*((const char **)a), *((const char **)b)); +} + /** * @brief Case-insensitive strstr() for platforms where strcasestr() diff --git a/src/rdstring.h b/src/rdstring.h index 67ea19401b..a036e6bff8 100644 --- a/src/rdstring.h +++ b/src/rdstring.h @@ -80,6 +80,10 @@ unsigned int rd_string_hash(const char *str, ssize_t len); int rd_strcmp(const char *a, const char *b); +int rd_strcmp2(const void *a, const void *b); + +int rd_strcmp3(const void *a, const void *b); + char *_rd_strcasestr(const char *haystack, const char *needle); char **rd_string_split(const char *input, diff --git a/tests/0022-consume_batch.c b/tests/0022-consume_batch.c index 64e826d035..ea7bdf1f2e 100644 --- a/tests/0022-consume_batch.c +++ b/tests/0022-consume_batch.c @@ -196,8 +196,70 @@ static void do_test_consume_batch_oauthbearer_cb(void) { #endif +/** + * @brief Subscribe to a non-existent topic with rd_kafka_consume_batch_queue. + * Verify that a rkmessage with error code ERR_UNKNOWN_TOPIC_OR_PART + * is received. + */ +static void do_test_consume_batch_non_existent_topic(void) { + + char *topic = "non-existent"; + rd_kafka_t *rk; + rd_kafka_topic_partition_list_t *rktpars; + rd_kafka_queue_t *rkq; + rd_kafka_message_t *rkms[1]; + rd_kafka_conf_t *conf; + ssize_t consumed = 0; + + SUB_TEST_QUICK(); + + test_conf_init(&conf, NULL, 30); + test_conf_set(conf, "allow.auto.create.topics", "false"); + test_conf_set(conf, "group.id", "test1"); + + /* Create simple consumer */ + rk = test_create_consumer(NULL, NULL, conf, NULL); + + /* Subscribe to the input topic */ + rktpars = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(rktpars, topic, + /* The partition is ignored in + * rd_kafka_subscribe() */ + RD_KAFKA_PARTITION_UA); + + rd_kafka_subscribe(rk, rktpars); + rd_kafka_topic_partition_list_destroy(rktpars); + + /* Create generic consume queue */ + rkq = rd_kafka_queue_get_consumer(rk); + + TEST_SAY("Consuming from non-existent topic\n"); + while ((consumed = rd_kafka_consume_batch_queue(rkq, 1000, rkms, 1)) != + 1) { + TEST_SAY("Consuming from non-existent topic\n"); + } + + TEST_ASSERT(rkms[0]->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART, + "Expected ERR_UNKNOWN_TOPIC_OR_PART, not %s: %s", + rd_kafka_err2str(rkms[0]->err), + rd_kafka_message_errstr(rkms[0])); + TEST_SAY("Received ERR_UNKNOWN_TOPIC_OR_PART\n"); + + TEST_SAY("Stopping consumer\n"); + + rd_kafka_message_destroy(rkms[0]); + + rd_kafka_queue_destroy(rkq); + + rd_kafka_destroy(rk); + + SUB_TEST_PASS(); +} + + int main_0022_consume_batch(int argc, char **argv) { do_test_consume_batch(); + do_test_consume_batch_non_existent_topic(); return 0; } diff --git a/tests/0033-regex_subscribe.c b/tests/0033-regex_subscribe.c index f31d33ebcb..07ae3d4a37 100644 --- a/tests/0033-regex_subscribe.c +++ b/tests/0033-regex_subscribe.c @@ -174,6 +174,13 @@ static void consumer_poll_once(rd_kafka_t *rk) { rkmessage->partition, rkmessage->offset); } else if (rkmessage->err == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART) { + /* Test segfault associated with this call is solved */ + int32_t leader_epoch = rd_kafka_message_leader_epoch(rkmessage); + TEST_ASSERT(leader_epoch == -1, + "rd_kafka_message_leader_epoch should be -1" + ", got %" PRId32, + leader_epoch); + if (strstr(rd_kafka_topic_name(rkmessage->rkt), "NONEXIST")) TEST_SAY("%s: %s: error is expected for this topic\n", rd_kafka_topic_name(rkmessage->rkt), diff --git a/tests/0045-subscribe_update.c b/tests/0045-subscribe_update.c index f804613d72..19462ab36a 100644 --- a/tests/0045-subscribe_update.c +++ b/tests/0045-subscribe_update.c @@ -36,6 +36,7 @@ * - topic additions * - topic deletions * - partition count changes + * - replica rack changes (using mock broker) */ @@ -142,6 +143,61 @@ static void await_no_rebalance(const char *pfx, rd_kafka_event_destroy(rkev); } + +/** + * Wait for REBALANCE event and perform assignment/unassignment. + * For the first time and after each event, wait till for \p timeout before + * stopping. Terminates earlier if \p min_events were seen. + * Asserts that \p min_events were processed. + * \p min_events set to 0 means it tries to drain all rebalance events and + * asserts only the fact that at least 1 event was processed. + */ +static void await_rebalance(const char *pfx, + rd_kafka_t *rk, + rd_kafka_queue_t *queue, + int timeout_ms, + int min_events) { + rd_kafka_event_t *rkev; + int processed = 0; + + while (1) { + TEST_SAY("%s: waiting for %d ms for rebalance event\n", pfx, + timeout_ms); + + rkev = test_wait_event(queue, RD_KAFKA_EVENT_REBALANCE, + timeout_ms); + if (!rkev) + break; + TEST_ASSERT(rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_REBALANCE, + "either expected a timeout or a " + "RD_KAFKA_EVENT_REBALANCE, got %s : %s", + rd_kafka_event_name(rkev), + rd_kafka_err2str(rd_kafka_event_error(rkev))); + + TEST_SAY("Calling test_rebalance_cb, assignment type is %s\n", + rd_kafka_rebalance_protocol(rk)); + test_rebalance_cb(rk, rd_kafka_event_error(rkev), + rd_kafka_event_topic_partition_list(rkev), + NULL); + + processed++; + + rd_kafka_event_destroy(rkev); + + if (min_events && processed >= min_events) + break; + } + + if (min_events) + min_events = 1; + + TEST_ASSERT( + processed >= min_events, + "Expected to process at least %d rebalance event, processed %d", + min_events, processed); +} + static void do_test_non_exist_and_partchange(void) { char *topic_a = rd_strdup(test_mk_topic_name("topic_a", 1)); rd_kafka_t *rk; @@ -421,6 +477,204 @@ static void do_test_regex_many_mock(const char *assignment_strategy, } +/** + * @brief Changing the broker racks should trigger a rejoin, if the client rack + * is set, and the set of partition racks changes due to the broker rack change. + * + * This is using the mock cluster. + * + */ +static void do_test_replica_rack_change_mock(const char *assignment_strategy, + rd_bool_t use_regex, + rd_bool_t use_client_rack, + rd_bool_t use_replica_rack) { + const char *subscription = use_regex ? "^top" : "topic"; + const char *topic = "topic"; + const char *test_name = tsprintf( + "Replica rack changes (%s, subscription = \"%s\", %s client.rack, " + "%s replica.rack)", + assignment_strategy, subscription, + use_client_rack ? "with" : "without", + use_replica_rack ? "with" : "without"); + rd_kafka_t *rk; + rd_kafka_conf_t *conf; + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_queue_t *queue; + + SUB_TEST("Testing %s", test_name); + + mcluster = test_mock_cluster_new(3, &bootstraps); + test_conf_init(&conf, NULL, 60 * 4); + + if (use_replica_rack) { + rd_kafka_mock_broker_set_rack(mcluster, 1, "rack0"); + rd_kafka_mock_broker_set_rack(mcluster, 2, "rack1"); + rd_kafka_mock_broker_set_rack(mcluster, 3, "rack2"); + } + + TEST_SAY("Creating topic %s\n", topic); + TEST_CALL_ERR__(rd_kafka_mock_topic_create(mcluster, topic, + 2 /* partition_cnt */, + 1 /* replication_factor */)); + + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "partition.assignment.strategy", + assignment_strategy); + /* Decrease metadata interval to speed up topic change discovery. */ + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "3000"); + + if (use_client_rack) + test_conf_set(conf, "client.rack", "client_rack"); + + rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_REBALANCE); + rk = test_create_consumer(test_str_id_generate_tmp(), NULL, conf, NULL); + queue = rd_kafka_queue_get_consumer(rk); + + TEST_SAY("%s: Subscribing via %s\n", test_name, subscription); + test_consumer_subscribe(rk, subscription); + + await_rebalance(tsprintf("%s: initial assignment", test_name), rk, + queue, 10000, 1); + + /* Avoid issues if the replica assignment algorithm for mock broker + * changes, and change all the racks. */ + if (use_replica_rack) { + TEST_SAY("%s: changing rack for all brokers\n", test_name); + rd_kafka_mock_broker_set_rack(mcluster, 1, "rack2"); + rd_kafka_mock_broker_set_rack(mcluster, 2, "rack0"); + rd_kafka_mock_broker_set_rack(mcluster, 3, "rack1"); + } + + if (use_client_rack && use_replica_rack) + await_rebalance(tsprintf("%s: rebalance", test_name), rk, queue, + 10000, 1); + else + await_no_rebalance( + tsprintf("%s: no rebalance without racks", test_name), rk, + queue, 10000); + + test_consumer_close(rk); + rd_kafka_queue_destroy(queue); + rd_kafka_destroy(rk); + test_mock_cluster_destroy(mcluster); + + SUB_TEST_PASS(); +} + + +/* Even if the leader has no rack, it should do rack-aware assignment in case + * one of the group members has a rack configured. */ +static void do_test_replica_rack_change_leader_no_rack_mock( + const char *assignment_strategy) { + const char *topic = "topic"; + const char *test_name = "Replica rack changes with leader rack absent."; + rd_kafka_t *c1, *c2; + rd_kafka_conf_t *conf1, *conf2; + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_queue_t *queue; + rd_kafka_topic_partition_list_t *asg1, *asg2; + + SUB_TEST("Testing %s", test_name); + + mcluster = test_mock_cluster_new(2, &bootstraps); + test_conf_init(&conf1, NULL, 60 * 4); + + rd_kafka_mock_broker_set_rack(mcluster, 1, "rack0"); + rd_kafka_mock_broker_set_rack(mcluster, 2, "rack1"); + + TEST_SAY("Creating topic %s\n", topic); + TEST_CALL_ERR__(rd_kafka_mock_topic_create(mcluster, topic, + 2 /* partition_cnt */, + 1 /* replication_factor */)); + + test_conf_set(conf1, "bootstrap.servers", bootstraps); + test_conf_set(conf1, "partition.assignment.strategy", + assignment_strategy); + /* Decrease metadata interval to speed up topic change discovery. */ + test_conf_set(conf1, "topic.metadata.refresh.interval.ms", "3000"); + + conf2 = rd_kafka_conf_dup(conf1); + + /* Setting the group.instance.id ensures that the leader is always c1. + */ + test_conf_set(conf1, "client.id", "client1Leader"); + test_conf_set(conf1, "group.instance.id", "client1Leader"); + + test_conf_set(conf2, "client.id", "client2Follower"); + test_conf_set(conf2, "group.instance.id", "client2Follower"); + test_conf_set(conf2, "client.rack", "rack0"); + + rd_kafka_conf_set_events(conf1, RD_KAFKA_EVENT_REBALANCE); + c1 = test_create_consumer("mygroup", NULL, conf1, NULL); + queue = rd_kafka_queue_get_consumer(c1); + + c2 = test_create_consumer("mygroup", NULL, conf2, NULL); + + TEST_SAY("%s: Subscribing via %s\n", test_name, topic); + test_consumer_subscribe(c1, topic); + test_consumer_subscribe(c2, topic); + + /* Poll to cause joining. */ + rd_kafka_poll(c1, 1); + rd_kafka_poll(c2, 1); + + /* Drain all events, as we want to process the assignment. */ + await_rebalance(tsprintf("%s: initial assignment", test_name), c1, + queue, 10000, 0); + + rd_kafka_assignment(c1, &asg1); + rd_kafka_assignment(c2, &asg2); + + /* Because of the deterministic nature of replica assignment in the mock + * broker, we can always be certain that topic:0 has its only replica on + * broker 1, and topic:1 has its only replica on broker 2. */ + TEST_ASSERT(asg1->cnt == 1 && asg1->elems[0].partition == 1, + "Expected c1 to be assigned topic1:1"); + TEST_ASSERT(asg2->cnt == 1 && asg2->elems[0].partition == 0, + "Expected c2 to be assigned topic1:0"); + + rd_kafka_topic_partition_list_destroy(asg1); + rd_kafka_topic_partition_list_destroy(asg2); + + /* Avoid issues if the replica assignment algorithm for mock broker + * changes, and change all the racks. */ + TEST_SAY("%s: changing rack for all brokers\n", test_name); + rd_kafka_mock_broker_set_rack(mcluster, 2, "rack0"); + rd_kafka_mock_broker_set_rack(mcluster, 1, "rack1"); + + /* Poll to cause rejoining. */ + rd_kafka_poll(c1, 1); + rd_kafka_poll(c2, 1); + + /* Drain all events, as we want to process the assignment. */ + await_rebalance(tsprintf("%s: rebalance", test_name), c1, queue, 10000, + 0); + + rd_kafka_assignment(c1, &asg1); + rd_kafka_assignment(c2, &asg2); + + /* Because of the deterministic nature of replica assignment in the mock + * broker, we can always be certain that topic:0 has its only replica on + * broker 1, and topic:1 has its only replica on broker 2. */ + TEST_ASSERT(asg1->cnt == 1 && asg1->elems[0].partition == 0, + "Expected c1 to be assigned topic1:0"); + TEST_ASSERT(asg2->cnt == 1 && asg2->elems[0].partition == 1, + "Expected c2 to be assigned topic1:1"); + + rd_kafka_topic_partition_list_destroy(asg1); + rd_kafka_topic_partition_list_destroy(asg2); + + test_consumer_close(c1); + test_consumer_close(c2); + rd_kafka_queue_destroy(queue); + rd_kafka_destroy(c1); + rd_kafka_destroy(c2); + test_mock_cluster_destroy(mcluster); + + SUB_TEST_PASS(); +} int main_0045_subscribe_update(int argc, char **argv) { @@ -457,3 +711,38 @@ int main_0045_subscribe_update_mock(int argc, char **argv) { return 0; } + + +int main_0045_subscribe_update_racks_mock(int argc, char **argv) { + int use_replica_rack = 0; + int use_client_rack = 0; + + if (test_needs_auth()) { + TEST_SKIP("Mock cluster does not support SSL/SASL\n"); + return 0; + } + + for (use_replica_rack = 0; use_replica_rack < 2; use_replica_rack++) { + for (use_client_rack = 0; use_client_rack < 2; + use_client_rack++) { + do_test_replica_rack_change_mock( + "range", rd_true /* use_regex */, use_client_rack, + use_replica_rack); + do_test_replica_rack_change_mock( + "range", rd_true /* use_regex */, use_client_rack, + use_replica_rack); + do_test_replica_rack_change_mock( + "cooperative-sticky", rd_true /* use_regex */, + use_client_rack, use_replica_rack); + do_test_replica_rack_change_mock( + "cooperative-sticky", rd_true /* use_regex */, + use_client_rack, use_replica_rack); + } + } + + /* Do not test with range assignor (yet) since it does not do rack aware + * assignment properly with the NULL rack, even for the Java client. */ + do_test_replica_rack_change_leader_no_rack_mock("cooperative-sticky"); + + return 0; +} diff --git a/tests/0089-max_poll_interval.c b/tests/0089-max_poll_interval.c index 3d7cbf66fa..908bc33474 100644 --- a/tests/0089-max_poll_interval.c +++ b/tests/0089-max_poll_interval.c @@ -351,8 +351,95 @@ static void do_test_with_log_queue(void) { SUB_TEST_PASS(); } + +/** + * @brief Consumer should be able to rejoin the group just by polling after + * leaving due to a max.poll.interval.ms timeout. The poll does not need to + * go through any special function, any queue containing consumer messages + * should suffice. + * We test with the result of rd_kafka_queue_get_consumer, and an arbitrary + * queue that is forwarded to by the result of rd_kafka_queue_get_consumer. + */ +static void +do_test_rejoin_after_interval_expire(rd_bool_t forward_to_another_q) { + const char *topic = test_mk_topic_name("0089_max_poll_interval", 1); + rd_kafka_conf_t *conf; + char groupid[64]; + rd_kafka_t *rk = NULL; + rd_kafka_queue_t *consumer_queue = NULL; + rd_kafka_event_t *event = NULL; + rd_kafka_queue_t *polling_queue = NULL; + + SUB_TEST("Testing with forward_to_another_q = %d", + forward_to_another_q); + + test_create_topic(NULL, topic, 1, 1); + + test_str_id_generate(groupid, sizeof(groupid)); + test_conf_init(&conf, NULL, 60); + test_conf_set(conf, "session.timeout.ms", "6000"); + test_conf_set(conf, "max.poll.interval.ms", "10000" /*10s*/); + test_conf_set(conf, "partition.assignment.strategy", "range"); + + /* We need to specify a non-NULL rebalance CB to get events of type + * RD_KAFKA_EVENT_REBALANCE. */ + rk = test_create_consumer(groupid, test_rebalance_cb, conf, NULL); + + consumer_queue = rd_kafka_queue_get_consumer(rk); + + test_consumer_subscribe(rk, topic); + + if (forward_to_another_q) { + polling_queue = rd_kafka_queue_new(rk); + rd_kafka_queue_forward(consumer_queue, polling_queue); + } else + polling_queue = consumer_queue; + + event = test_wait_event(polling_queue, RD_KAFKA_EVENT_REBALANCE, + (int)(test_timeout_multiplier * 10000)); + TEST_ASSERT(event, + "Did not get a rebalance event for initial group join"); + TEST_ASSERT(rd_kafka_event_error(event) == + RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, + "Group join should assign partitions"); + rd_kafka_assign(rk, rd_kafka_event_topic_partition_list(event)); + rd_kafka_event_destroy(event); + + rd_sleep(10 + 1); /* Exceed max.poll.interval.ms. */ + + /* Note that by polling for the group leave, we're also polling the + * consumer queue, and hence it should trigger a rejoin. */ + event = test_wait_event(polling_queue, RD_KAFKA_EVENT_REBALANCE, + (int)(test_timeout_multiplier * 10000)); + TEST_ASSERT(event, "Did not get a rebalance event for the group leave"); + TEST_ASSERT(rd_kafka_event_error(event) == + RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS, + "Group leave should revoke partitions"); + rd_kafka_assign(rk, NULL); + rd_kafka_event_destroy(event); + + event = test_wait_event(polling_queue, RD_KAFKA_EVENT_REBALANCE, + (int)(test_timeout_multiplier * 10000)); + TEST_ASSERT(event, "Should get a rebalance event for the group rejoin"); + TEST_ASSERT(rd_kafka_event_error(event) == + RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, + "Group rejoin should assign partitions"); + rd_kafka_assign(rk, rd_kafka_event_topic_partition_list(event)); + rd_kafka_event_destroy(event); + + if (forward_to_another_q) + rd_kafka_queue_destroy(polling_queue); + rd_kafka_queue_destroy(consumer_queue); + test_consumer_close(rk); + rd_kafka_destroy(rk); + + SUB_TEST_PASS(); +} + int main_0089_max_poll_interval(int argc, char **argv) { do_test(); do_test_with_log_queue(); + do_test_rejoin_after_interval_expire(rd_false); + do_test_rejoin_after_interval_expire(rd_true); return 0; } diff --git a/tests/0104-fetch_from_follower_mock.c b/tests/0104-fetch_from_follower_mock.c index 1ecf99da3f..11460beaf0 100644 --- a/tests/0104-fetch_from_follower_mock.c +++ b/tests/0104-fetch_from_follower_mock.c @@ -595,6 +595,8 @@ int main_0104_fetch_from_follower_mock(int argc, char **argv) { return 0; } + test_timeout_set(50); + do_test_offset_reset("earliest"); do_test_offset_reset("latest"); diff --git a/tests/0130-store_offsets.c b/tests/0130-store_offsets.c index 9fb8d2350a..f06f31f3ac 100644 --- a/tests/0130-store_offsets.c +++ b/tests/0130-store_offsets.c @@ -30,8 +30,8 @@ /** - * Verify that offsets_store() is not allowed for unassigned partitions, - * and that those offsets are not committed. + * Verify that offsets_store() commits the right offsets and metadata, + * and is not allowed for unassigned partitions. */ static void do_test_store_unassigned(void) { const char *topic = test_mk_topic_name("0130_store_unassigned", 1); @@ -40,6 +40,7 @@ static void do_test_store_unassigned(void) { rd_kafka_topic_partition_list_t *parts; rd_kafka_resp_err_t err; rd_kafka_message_t *rkmessage; + char metadata[] = "metadata"; const int64_t proper_offset = 900, bad_offset = 300; SUB_TEST_QUICK(); @@ -60,8 +61,13 @@ static void do_test_store_unassigned(void) { TEST_SAY("Consume one message\n"); test_consumer_poll_once(c, NULL, tmout_multip(3000)); - parts->elems[0].offset = proper_offset; - TEST_SAY("Storing offset %" PRId64 " while assigned: should succeed\n", + parts->elems[0].offset = proper_offset; + parts->elems[0].metadata_size = sizeof metadata; + parts->elems[0].metadata = malloc(parts->elems[0].metadata_size); + memcpy(parts->elems[0].metadata, metadata, + parts->elems[0].metadata_size); + TEST_SAY("Storing offset %" PRId64 + " with metadata while assigned: should succeed\n", parts->elems[0].offset); TEST_CALL_ERR__(rd_kafka_offsets_store(c, parts)); @@ -71,7 +77,10 @@ static void do_test_store_unassigned(void) { TEST_SAY("Unassigning partitions and trying to store again\n"); TEST_CALL_ERR__(rd_kafka_assign(c, NULL)); - parts->elems[0].offset = bad_offset; + parts->elems[0].offset = bad_offset; + parts->elems[0].metadata_size = 0; + rd_free(parts->elems[0].metadata); + parts->elems[0].metadata = NULL; TEST_SAY("Storing offset %" PRId64 " while unassigned: should fail\n", parts->elems[0].offset); err = rd_kafka_offsets_store(c, parts); @@ -108,9 +117,50 @@ static void do_test_store_unassigned(void) { "offset %" PRId64 ", not %" PRId64, proper_offset, rkmessage->offset); + TEST_SAY( + "Retrieving committed offsets to verify committed offset " + "metadata\n"); + rd_kafka_topic_partition_list_t *committed_toppar; + committed_toppar = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(committed_toppar, topic, 0); + TEST_CALL_ERR__( + rd_kafka_committed(c, committed_toppar, tmout_multip(3000))); + TEST_ASSERT(committed_toppar->elems[0].offset == proper_offset, + "Expected committed offset to be %" PRId64 ", not %" PRId64, + proper_offset, committed_toppar->elems[0].offset); + TEST_ASSERT(committed_toppar->elems[0].metadata != NULL, + "Expected metadata to not be NULL"); + TEST_ASSERT(strcmp(committed_toppar->elems[0].metadata, metadata) == 0, + "Expected metadata to be %s, not %s", metadata, + (char *)committed_toppar->elems[0].metadata); + + TEST_SAY("Storing next offset without metadata\n"); + parts->elems[0].offset = proper_offset + 1; + TEST_CALL_ERR__(rd_kafka_offsets_store(c, parts)); + + TEST_SAY("Committing\n"); + TEST_CALL_ERR__(rd_kafka_commit(c, NULL, rd_false /*sync*/)); + + TEST_SAY( + "Retrieving committed offset to verify empty committed offset " + "metadata\n"); + rd_kafka_topic_partition_list_t *committed_toppar_empty; + committed_toppar_empty = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(committed_toppar_empty, topic, 0); + TEST_CALL_ERR__( + rd_kafka_committed(c, committed_toppar_empty, tmout_multip(3000))); + TEST_ASSERT(committed_toppar_empty->elems[0].offset == + proper_offset + 1, + "Expected committed offset to be %" PRId64 ", not %" PRId64, + proper_offset, committed_toppar_empty->elems[0].offset); + TEST_ASSERT(committed_toppar_empty->elems[0].metadata == NULL, + "Expected metadata to be NULL"); + rd_kafka_message_destroy(rkmessage); rd_kafka_topic_partition_list_destroy(parts); + rd_kafka_topic_partition_list_destroy(committed_toppar); + rd_kafka_topic_partition_list_destroy(committed_toppar_empty); rd_kafka_consumer_close(c); rd_kafka_destroy(c); diff --git a/tests/0139-offset_validation_mock.c b/tests/0139-offset_validation_mock.c new file mode 100644 index 0000000000..3fff5277a4 --- /dev/null +++ b/tests/0139-offset_validation_mock.c @@ -0,0 +1,151 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2023, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#include "test.h" + +#include "../src/rdkafka_proto.h" + + +struct _produce_args { + const char *topic; + int sleep; + rd_kafka_conf_t *conf; +}; + +static int produce_concurrent_thread(void *args) { + rd_kafka_t *p1; + test_curr->exp_dr_err = RD_KAFKA_RESP_ERR_NO_ERROR; + test_curr->exp_dr_status = RD_KAFKA_MSG_STATUS_PERSISTED; + + struct _produce_args *produce_args = args; + rd_sleep(produce_args->sleep); + + p1 = test_create_handle(RD_KAFKA_PRODUCER, produce_args->conf); + TEST_CALL_ERR__( + rd_kafka_producev(p1, RD_KAFKA_V_TOPIC(produce_args->topic), + RD_KAFKA_V_VALUE("hi", 2), RD_KAFKA_V_END)); + rd_kafka_flush(p1, -1); + rd_kafka_destroy(p1); + return 0; +} + +/** + * @brief Send a produce request in the middle of an offset validation + * and expect that the fetched message is discarded, don't producing + * a duplicate when state becomes active again. See #4249. + */ +static void do_test_no_duplicates_during_offset_validation(void) { + const char *topic = test_mk_topic_name(__FUNCTION__, 1); + const char *c1_groupid = topic; + rd_kafka_t *c1; + rd_kafka_conf_t *conf, *conf_producer; + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + int initial_msg_count = 5; + thrd_t thrd; + struct _produce_args args = RD_ZERO_INIT; + uint64_t testid = test_id_generate(); + + SUB_TEST_QUICK(); + + mcluster = test_mock_cluster_new(1, &bootstraps); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + + /* Slow down OffsetForLeaderEpoch so a produce and + * subsequent fetch can happen while it's in-flight */ + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_OffsetForLeaderEpoch, 1, + RD_KAFKA_RESP_ERR_NO_ERROR, 5000); + + test_conf_init(&conf_producer, NULL, 60); + test_conf_set(conf_producer, "bootstrap.servers", bootstraps); + + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, testid, 0, 0, initial_msg_count, 10, + "bootstrap.servers", bootstraps, + "batch.num.messages", "1", NULL); + + args.topic = topic; + /* Makes that the message is produced while an offset validation + * is ongoing */ + args.sleep = 5; + args.conf = conf_producer; + /* Spin up concurrent thread */ + if (thrd_create(&thrd, produce_concurrent_thread, (void *)&args) != + thrd_success) + TEST_FAIL("Failed to create thread"); + + test_conf_init(&conf, NULL, 60); + + test_conf_set(conf, "bootstrap.servers", bootstraps); + /* Makes that an offset validation happens at the same + * time a new message is being produced */ + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + test_conf_set(conf, "enable.auto.commit", "false"); + test_conf_set(conf, "enable.auto.offset.store", "false"); + test_conf_set(conf, "enable.partition.eof", "true"); + + c1 = test_create_consumer(c1_groupid, NULL, conf, NULL); + test_consumer_subscribe(c1, topic); + + /* Consume initial messages */ + test_consumer_poll("MSG_INIT", c1, testid, 0, 0, initial_msg_count, + NULL); + /* EOF after initial messages */ + test_consumer_poll("MSG_EOF", c1, testid, 1, initial_msg_count, 0, + NULL); + /* Concurrent producer message and EOF */ + test_consumer_poll("MSG_AND_EOF", c1, testid, 1, initial_msg_count, 1, + NULL); + /* Only an EOF, not a duplicate message */ + test_consumer_poll("MSG_EOF2", c1, testid, 1, initial_msg_count, 0, + NULL); + + thrd_join(thrd, NULL); + + rd_kafka_destroy(c1); + + test_mock_cluster_destroy(mcluster); + + TEST_LATER_CHECK(); + SUB_TEST_PASS(); +} + +int main_0139_offset_validation_mock(int argc, char **argv) { + + if (test_needs_auth()) { + TEST_SKIP("Mock cluster does not support SSL/SASL\n"); + return 0; + } + + do_test_no_duplicates_during_offset_validation(); + + return 0; +} diff --git a/tests/0140-commit_metadata.cpp b/tests/0140-commit_metadata.cpp new file mode 100644 index 0000000000..fae655915b --- /dev/null +++ b/tests/0140-commit_metadata.cpp @@ -0,0 +1,108 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2023, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + + +#include "testcpp.h" + +using namespace std; + +/** + * @brief Committed metadata should be stored and received back when + * checking committed offsets. + */ +static void test_commit_metadata() { + SUB_TEST_QUICK(); + + std::string bootstraps; + std::string errstr; + RdKafka::ErrorCode err; + + RdKafka::Conf *conf; + std::string topic = Test::mk_topic_name(__FUNCTION__, 1); + Test::conf_init(&conf, NULL, 3000); + Test::conf_set(conf, "group.id", topic); + + RdKafka::KafkaConsumer *consumer = + RdKafka::KafkaConsumer::create(conf, errstr); + if (!consumer) + Test::Fail("Failed to create KafkaConsumer: " + errstr); + delete conf; + + Test::Say("Create topic.\n"); + Test::create_topic(consumer, topic.c_str(), 1, 1); + + Test::Say("Commit offsets.\n"); + std::vector offsets; + RdKafka::TopicPartition *offset = + RdKafka::TopicPartition::create(topic, 0, 10); + + std::string metadata = "some_metadata"; + std::vector metadata_vect(metadata.begin(), metadata.end()); + + offset->set_metadata(metadata_vect); + offsets.push_back(offset); + + err = consumer->commitSync(offsets); + TEST_ASSERT(!err, "commit failed: %s", RdKafka::err2str(err).c_str()); + RdKafka::TopicPartition::destroy(offsets); + + Test::Say("Read committed offsets.\n"); + offset = RdKafka::TopicPartition::create(topic, 0, 10); + offsets.push_back(offset); + err = consumer->committed(offsets, 5000); + TEST_ASSERT(!err, "committed offsets failed: %s", + RdKafka::err2str(err).c_str()); + TEST_ASSERT(offsets.size() == 1, "expected offsets size 1, got %" PRIusz, + offsets.size()); + + Test::Say("Check committed metadata.\n"); + std::vector metadata_vect_committed = + offsets[0]->get_metadata(); + std::string metadata_committed(metadata_vect_committed.begin(), + metadata_vect_committed.end()); + + if (metadata != metadata_committed) { + Test::Fail(tostr() << "Expecting metadata to be \"" << metadata + << "\", got \"" << metadata_committed << "\""); + } + + RdKafka::TopicPartition::destroy(offsets); + + consumer->close(); + + delete consumer; + + SUB_TEST_PASS(); +} + +extern "C" { +int main_0140_commit_metadata(int argc, char **argv) { + test_commit_metadata(); + return 0; +} +} diff --git a/tests/8001-fetch_from_follower_mock_manual.c b/tests/8001-fetch_from_follower_mock_manual.c new file mode 100644 index 0000000000..d542be5f91 --- /dev/null +++ b/tests/8001-fetch_from_follower_mock_manual.c @@ -0,0 +1,116 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2023, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#include "test.h" + +#include "../src/rdkafka_proto.h" + +/** + * @brief Test that the #4195 segfault doesn't happen when preferred replica + * lease expires and the rktp is in fetch state + * RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT. + */ +static void do_test_fetch_from_follower_offset_retry(void) { + const char *bootstraps; + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + rd_kafka_t *c; + const char *topic = "test"; + rd_kafka_topic_partition_t *rktpar; + rd_kafka_topic_partition_list_t *seek; + int i; + + SUB_TEST_QUICK(); + test_timeout_set(600); + + 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, "fetch.error.backoff.ms", "1000"); + test_conf_set(conf, "fetch.message.max.bytes", "10"); + test_conf_set(conf, "session.timeout.ms", "600000"); + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "600000"); + + c = test_create_consumer("mygroup", NULL, conf, NULL); + + test_consumer_assign_partition( + "do_test_fetch_from_follower_offset_retry", 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. + * Subsequent polls are for waiting up to 5 minutes. */ + for (i = 0; i < 7; i++) { + test_consumer_poll_no_msgs( + "initial metadata and preferred replica set", c, 0, 40000); + } + + + /* Seek to end to trigger ListOffsets */ + seek = rd_kafka_topic_partition_list_new(1); + rktpar = rd_kafka_topic_partition_list_add(seek, topic, 0); + rktpar->offset = RD_KAFKA_OFFSET_END; + + /* Increase RTT for this ListOffsets */ + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 2, RD_KAFKAP_ListOffsets, 1, RD_KAFKA_RESP_ERR_NO_ERROR, + 40 * 1000); + + rd_kafka_seek_partitions(c, seek, -1); + rd_kafka_topic_partition_list_destroy(seek); + + /* Wait lease expiry */ + rd_sleep(50); + + test_consumer_close(c); + + rd_kafka_destroy(c); + + test_mock_cluster_destroy(mcluster); + + SUB_TEST_PASS(); +} + + +int main_8001_fetch_from_follower_mock_manual(int argc, char **argv) { + + if (test_needs_auth()) { + TEST_SKIP("Mock cluster does not support SSL/SASL\n"); + return 0; + } + + do_test_fetch_from_follower_offset_retry(); + + return 0; +} diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index a9dccfa5e5..9e74df8fee 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -129,7 +129,10 @@ set( 0136-resolve_cb.c 0137-barrier_batch_consume.c 0138-admin_mock.c + 0139-offset_validation_mock.c + 0140-commit_metadata.cpp 8000-idle.cpp + 8001-fetch_from_follower_mock_manual.c test.c testcpp.cpp rusage.c diff --git a/tests/Makefile b/tests/Makefile index 73eab21406..543639e49b 100644 --- a/tests/Makefile +++ b/tests/Makefile @@ -12,7 +12,7 @@ CXXFLAGS += -I../src -I../src-cpp LDFLAGS += -rdynamic -L../src -L../src-cpp # Latest Kafka version -KAFKA_VERSION?=3.1.0 +KAFKA_VERSION?=3.4.0 # Kafka versions for compatibility tests COMPAT_KAFKA_VERSIONS?=0.8.2.2 0.9.0.1 0.11.0.3 1.0.2 2.4.1 2.8.1 $(KAFKA_VERSION) diff --git a/tests/test.c b/tests/test.c index 71180c8f47..0aa250c03f 100644 --- a/tests/test.c +++ b/tests/test.c @@ -152,6 +152,7 @@ _TEST_DECL(0045_subscribe_update); _TEST_DECL(0045_subscribe_update_topic_remove); _TEST_DECL(0045_subscribe_update_non_exist_and_partchange); _TEST_DECL(0045_subscribe_update_mock); +_TEST_DECL(0045_subscribe_update_racks_mock); _TEST_DECL(0046_rkt_cache); _TEST_DECL(0047_partial_buf_tmout); _TEST_DECL(0048_partitioner); @@ -246,9 +247,13 @@ _TEST_DECL(0135_sasl_credentials); _TEST_DECL(0136_resolve_cb); _TEST_DECL(0137_barrier_batch_consume); _TEST_DECL(0138_admin_mock); +_TEST_DECL(0139_offset_validation_mock); +_TEST_DECL(0140_commit_metadata); + /* Manual tests */ _TEST_DECL(8000_idle); +_TEST_DECL(8001_fetch_from_follower_mock_manual); /* Define test resource usage thresholds if the default limits @@ -359,6 +364,7 @@ struct test tests[] = { TEST_BRKVER(0, 9, 0, 0), .scenario = "noautocreate"), _TEST(0045_subscribe_update_mock, TEST_F_LOCAL), + _TEST(0045_subscribe_update_racks_mock, TEST_F_LOCAL), _TEST(0046_rkt_cache, TEST_F_LOCAL), _TEST(0047_partial_buf_tmout, TEST_F_KNOWN_ISSUE), _TEST(0048_partitioner, @@ -490,9 +496,12 @@ struct test tests[] = { _TEST(0136_resolve_cb, TEST_F_LOCAL), _TEST(0137_barrier_batch_consume, 0), _TEST(0138_admin_mock, TEST_F_LOCAL, TEST_BRKVER(2, 4, 0, 0)), + _TEST(0139_offset_validation_mock, 0), + _TEST(0140_commit_metadata, 0), /* Manual tests */ _TEST(8000_idle, TEST_F_MANUAL), + _TEST(8001_fetch_from_follower_mock_manual, TEST_F_MANUAL), {NULL}}; diff --git a/vcpkg.json b/vcpkg.json index 5e446107a6..6d550efe9c 100644 --- a/vcpkg.json +++ b/vcpkg.json @@ -1,6 +1,6 @@ { "name": "librdkafka", - "version": "2.1.0", + "version": "2.1.1", "dependencies": [ { "name": "zstd", diff --git a/win32/tests/tests.vcxproj b/win32/tests/tests.vcxproj index de69a62d85..09eb81f0d6 100644 --- a/win32/tests/tests.vcxproj +++ b/win32/tests/tests.vcxproj @@ -219,7 +219,10 @@ + + +