Skip to content

Commit

Permalink
Fix duplicate messages when resuming partitions not paused (#4636)
Browse files Browse the repository at this point in the history
In case of subscription change with a consumer using the cooperative assignor
it could resume fetching from a previous position.
That could also happen if resuming a partition that wasn't paused.
Fixed by ensuring that a resume operation is completely a no-op when
the partition isn't paused
  • Loading branch information
emasab authored and anchitj committed Jun 10, 2024
1 parent 5db871a commit 42e64ae
Show file tree
Hide file tree
Showing 7 changed files with 171 additions and 4 deletions.
10 changes: 10 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@ librdkafka v2.3.1 is a maintenance release:
* Fix pipeline inclusion of static binaries (#4666)
* Fix to main loop timeout calculation leading to a tight loop for a
max period of 1 ms (#4671).
* Fixed a bug causing duplicate message consumption from a stale
fetch start offset in some particular cases (#4636)


## Fixes
Expand All @@ -27,6 +29,14 @@ librdkafka v2.3.1 is a maintenance release:
leading to increased CPU usage until the timeout was reached.
Happening since 1.x (#4671).

### Consumer fixes

* In case of subscription change with a consumer using the cooperative assignor
it could resume fetching from a previous position.
That could also happen if resuming a partition that wasn't paused.
Fixed by ensuring that a resume operation is completely a no-op when
the partition isn't paused (#4636).



# librdkafka v2.3.0
Expand Down
18 changes: 16 additions & 2 deletions src/rdkafka_partition.c
Original file line number Diff line number Diff line change
Expand Up @@ -2299,7 +2299,22 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_pause_resume(rd_kafka_toppar_t *rktp,
int flag,
rd_kafka_replyq_t replyq) {
int32_t version;
rd_kafka_op_t *rko;
rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_PAUSE);

if (!pause) {
/* If partitions isn't paused, avoid bumping its version,
* as it'll result in resuming fetches from a stale
* next_fetch_start */
rd_bool_t is_paused = rd_false;
rd_kafka_toppar_lock(rktp);
is_paused = RD_KAFKA_TOPPAR_IS_PAUSED(rktp);
rd_kafka_toppar_unlock(rktp);
if (!is_paused) {
rko->rko_replyq = replyq;
rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR_NO_ERROR);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
}

/* Bump version barrier. */
version = rd_kafka_toppar_version_new_barrier(rktp);
Expand All @@ -2310,7 +2325,6 @@ rd_kafka_resp_err_t rd_kafka_toppar_op_pause_resume(rd_kafka_toppar_t *rktp,
RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
rktp->rktp_partition, version);

rko = rd_kafka_op_new(RD_KAFKA_OP_PAUSE);
rko->rko_version = version;
rko->rko_u.pause.pause = pause;
rko->rko_u.pause.flag = flag;
Expand Down
24 changes: 22 additions & 2 deletions tests/0050-subscribe_adds.c
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
* 2023, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
Expand All @@ -28,6 +29,7 @@

#include "test.h"
#include "rdkafka.h"
#include "../src/rdkafka_proto.h"

#include <stdarg.h>

Expand All @@ -41,9 +43,13 @@
* * Verify that all messages from all three topics are consumed
* * Subscribe to T1,T3
* * Verify that there were no duplicate messages.
*
* @param partition_assignment_strategy Assignment strategy to test.
*/
static void
test_no_duplicate_messages(const char *partition_assignment_strategy) {

int main_0050_subscribe_adds(int argc, char **argv) {
SUB_TEST("%s", partition_assignment_strategy);
rd_kafka_t *rk;
#define TOPIC_CNT 3
char *topic[TOPIC_CNT] = {
Expand Down Expand Up @@ -80,6 +86,8 @@ int main_0050_subscribe_adds(int argc, char **argv) {

test_conf_init(&conf, &tconf, 60);
test_topic_conf_set(tconf, "auto.offset.reset", "smallest");
test_conf_set(conf, "partition.assignment.strategy",
partition_assignment_strategy);

rk = test_create_consumer(topic[0], NULL, conf, tconf);

Expand All @@ -105,7 +113,7 @@ int main_0050_subscribe_adds(int argc, char **argv) {
err = rd_kafka_subscribe(rk, tlist);
TEST_ASSERT(!err, "subscribe() failed: %s", rd_kafka_err2str(err));

test_consumer_poll_no_msgs("consume", rk, testid, (int)(6000 * 1.5));
test_consumer_poll_no_msgs("consume", rk, testid, (int)(3000));


test_msgver_verify("consume", &mv, TEST_MSGVER_ORDER | TEST_MSGVER_DUP,
Expand All @@ -120,5 +128,17 @@ int main_0050_subscribe_adds(int argc, char **argv) {
for (i = 0; i < TOPIC_CNT; i++)
rd_free(topic[i]);

SUB_TEST_PASS();
#undef TOPIC_CNT
}

int main_0050_subscribe_adds(int argc, char **argv) {

test_no_duplicate_messages("range");

test_no_duplicate_messages("roundrobin");

test_no_duplicate_messages("cooperative-sticky");

return 0;
}
119 changes: 119 additions & 0 deletions tests/0145-pause_resume_mock.c
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2024, 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 "rdkafka.h"
#include "../src/rdkafka_proto.h"

#include <stdarg.h>

/**
* Verify that no duplicate message are consumed after an unnecessary
* resume, ensuring the fetch version isn't bumped, leading to
* using a stale next fetch start.
*
* @param partition_assignment_strategy Assignment strategy to test.
*/
static void test_no_duplicate_messages_unnecessary_resume(
const char *partition_assignment_strategy) {
const char *bootstraps;
rd_kafka_mock_cluster_t *mcluster;
rd_kafka_conf_t *conf;
rd_kafka_topic_conf_t *tconf;
rd_kafka_t *rk;
test_msgver_t mv;
rd_kafka_topic_partition_list_t *tlist;
char *topic =
rd_strdup(test_mk_topic_name("0050_unnecessary_resume_1", 1));
uint64_t testid = test_id_generate();
int msgcnt = 100;

SUB_TEST("%s", partition_assignment_strategy);

mcluster = test_mock_cluster_new(3, &bootstraps);
rd_kafka_mock_topic_create(mcluster, topic, 1, 1);

TEST_SAY("Seed the topic with messages\n");
test_produce_msgs_easy_v(topic, testid, RD_KAFKA_PARTITION_UA, 0,
msgcnt, 1000, "bootstrap.servers", bootstraps,
NULL);

test_conf_init(&conf, &tconf, 60);
test_topic_conf_set(tconf, "auto.offset.reset", "smallest");
test_conf_set(conf, "bootstrap.servers", bootstraps);
test_conf_set(conf, "partition.assignment.strategy",
partition_assignment_strategy);

TEST_SAY("Subscribe to topic\n");
tlist = rd_kafka_topic_partition_list_new(1);
rd_kafka_topic_partition_list_add(tlist, topic, RD_KAFKA_PARTITION_UA);

rk = test_create_consumer("mygroup", NULL, conf, tconf);
TEST_CALL_ERR__(rd_kafka_subscribe(rk, tlist));

TEST_SAY("Consume and verify messages\n");
test_msgver_init(&mv, testid);
test_consumer_poll("consume", rk, testid, -1, 0, msgcnt, &mv);

TEST_SAY("Unnecessary resume\n");
tlist->elems[0].partition = 0; /* Resume the only partition */
TEST_CALL_ERR__(rd_kafka_resume_partitions(rk, tlist));

TEST_SAY("Ensure no duplicate messages\n");
test_consumer_poll_no_msgs("consume", rk, testid, (int)(3000));

test_msgver_verify("consume", &mv, TEST_MSGVER_ORDER | TEST_MSGVER_DUP,
0, msgcnt);

test_msgver_clear(&mv);

rd_kafka_topic_partition_list_destroy(tlist);
rd_kafka_consumer_close(rk);
rd_kafka_destroy(rk);

test_mock_cluster_destroy(mcluster);

rd_free(topic);

SUB_TEST_PASS();
}

int main_0145_pause_resume_mock(int argc, char **argv) {
if (test_needs_auth()) {
TEST_SAY("Mock cluster does not support SSL/SASL\n");
return 0;
}

test_no_duplicate_messages_unnecessary_resume("range");

test_no_duplicate_messages_unnecessary_resume("roundrobin");

test_no_duplicate_messages_unnecessary_resume("cooperative-sticky");

return 0;
}
1 change: 1 addition & 0 deletions tests/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,7 @@ set(
0142-reauthentication.c
0143-exponential_backoff_mock.c
0144-idempotence_mock.c
0145-pause_resume_mock.c
8000-idle.cpp
8001-fetch_from_follower_mock_manual.c
test.c
Expand Down
2 changes: 2 additions & 0 deletions tests/test.c
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,7 @@ _TEST_DECL(0140_commit_metadata);
_TEST_DECL(0142_reauthentication);
_TEST_DECL(0143_exponential_backoff_mock);
_TEST_DECL(0144_idempotence_mock);
_TEST_DECL(0145_pause_resume_mock);

/* Manual tests */
_TEST_DECL(8000_idle);
Expand Down Expand Up @@ -511,6 +512,7 @@ struct test tests[] = {
_TEST(0142_reauthentication, 0, TEST_BRKVER(2, 2, 0, 0)),
_TEST(0143_exponential_backoff_mock, TEST_F_LOCAL),
_TEST(0144_idempotence_mock, TEST_F_LOCAL, TEST_BRKVER(0, 11, 0, 0)),
_TEST(0145_pause_resume_mock, TEST_F_LOCAL),


/* Manual tests */
Expand Down
1 change: 1 addition & 0 deletions win32/tests/tests.vcxproj
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,7 @@
<ClCompile Include="..\..\tests\0142-reauthentication.c" />
<ClCompile Include="..\..\tests\0143-exponential_backoff_mock.c" />
<ClCompile Include="..\..\tests\0144-idempotence_mock.c" />
<ClCompile Include="..\..\tests\0145-pause_resume_mock.c" />
<ClCompile Include="..\..\tests\8000-idle.cpp" />
<ClCompile Include="..\..\tests\8001-fetch_from_follower_mock_manual.c" />
<ClCompile Include="..\..\tests\test.c" />
Expand Down

0 comments on commit 42e64ae

Please sign in to comment.