RdKafka::ErrorCode RdKafka::HandleImpl::pause (std::vector<RdKafka::TopicPartition*> &partitions) { rd_kafka_topic_partition_list_t *c_parts; rd_kafka_resp_err_t err; c_parts = partitions_to_c_parts(partitions); err = rd_kafka_pause_partitions(rk_, c_parts); if (!err) update_partitions_from_c_parts(partitions, c_parts); rd_kafka_topic_partition_list_destroy(c_parts); return static_cast<RdKafka::ErrorCode>(err); }
static PyObject *Consumer_pause(Handle *self, PyObject *args, PyObject *kwargs) { PyObject *plist; rd_kafka_topic_partition_list_t *c_parts; rd_kafka_resp_err_t err; static char *kws[] = {"partitions", NULL}; if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O", kws, &plist)) return NULL; if (!(c_parts = py_to_c_parts(plist))) return NULL; err = rd_kafka_pause_partitions(self->rk, c_parts); rd_kafka_topic_partition_list_destroy(c_parts); if (err) { cfl_PyErr_Format(err, "Failed to pause partitions: %s", rd_kafka_err2str(err)); return NULL; } Py_RETURN_NONE; }
static void do_offset_test (const char *what, int auto_commit, int auto_store, int async) { test_timing_t t_all; char groupid[64]; rd_kafka_t *rk; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *tconf; int cnt = 0; const int extra_cnt = 5; rd_kafka_resp_err_t err; rd_kafka_topic_partition_list_t *parts; rd_kafka_topic_partition_t *rktpar; int64_t next_offset = -1; test_conf_init(&conf, &tconf, 20); test_conf_set(conf, "enable.auto.commit", auto_commit ? "true":"false"); test_conf_set(conf, "enable.auto.offset.store", auto_store ?"true":"false"); test_conf_set(conf, "auto.commit.interval.ms", "500"); rd_kafka_conf_set_offset_commit_cb(conf, offset_commit_cb); test_topic_conf_set(tconf, "auto.offset.reset", "smallest"); test_str_id_generate(groupid, sizeof(groupid)); test_conf_set(conf, "group.id", groupid); rd_kafka_conf_set_default_topic_conf(conf, tconf); TEST_SAY(_C_MAG "[ do_offset_test: %s with group.id %s ]\n", what, groupid); TIMING_START(&t_all, what); expected_offset = 0; committed_offset = -1; /* MO: * - Create consumer. * - Start consuming from beginning * - Perform store & commits according to settings * - Stop storing&committing when half of the messages are consumed, * - but consume 5 more to check against. * - Query position. * - Destroy consumer. * - Create new consumer with same group.id using stored offsets * - Should consume the expected message. */ /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, rd_kafka_conf_dup(conf)); rd_kafka_poll_set_consumer(rk); test_consumer_subscribe(rk, topic); while (cnt - extra_cnt < msgcnt / 2) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(rk, 10*1000); if (!rkm) continue; if (rkm->err == RD_KAFKA_RESP_ERR__TIMED_OUT) TEST_FAIL("%s: Timed out waiting for message %d", what,cnt); else if (rkm->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { rd_kafka_message_destroy(rkm); continue; } else if (rkm->err) TEST_FAIL("%s: Consumer error: %s", what, rd_kafka_message_errstr(rkm)); /* Offset of next message. */ next_offset = rkm->offset + 1; if (cnt < msgcnt / 2) { if (!auto_store) { err = rd_kafka_offset_store(rkm->rkt,rkm->partition, rkm->offset); if (err) TEST_FAIL("%s: offset_store failed: %s\n", what, rd_kafka_err2str(err)); } expected_offset = rkm->offset+1; if (!auto_commit) { test_timing_t t_commit; TIMING_START(&t_commit, async?"commit.async":"commit.sync"); err = rd_kafka_commit_message(rk, rkm, async); TIMING_STOP(&t_commit); if (err) TEST_FAIL("%s: commit failed: %s\n", what, rd_kafka_err2str(err)); } } else if (auto_store && auto_commit) expected_offset = rkm->offset+1; rd_kafka_message_destroy(rkm); cnt++; } TEST_SAY("%s: done consuming after %d messages, at offset %"PRId64"\n", what, cnt, expected_offset); if ((err = rd_kafka_assignment(rk, &parts))) TEST_FAIL("%s: failed to get assignment(): %s\n", what, rd_kafka_err2str(err)); /* Verify position */ if ((err = rd_kafka_position(rk, parts))) TEST_FAIL("%s: failed to get position(): %s\n", what, rd_kafka_err2str(err)); if (!(rktpar = rd_kafka_topic_partition_list_find(parts, topic, partition))) TEST_FAIL("%s: position(): topic lost\n", what); if (rktpar->offset != next_offset) TEST_FAIL("%s: Expected position() offset %"PRId64", got %"PRId64, what, next_offset, rktpar->offset); TEST_SAY("%s: Position is at %"PRId64", good!\n", what, rktpar->offset); /* Pause messages while waiting so we can serve callbacks * without having more messages received. */ if ((err = rd_kafka_pause_partitions(rk, parts))) TEST_FAIL("%s: failed to pause partitions: %s\n", what, rd_kafka_err2str(err)); rd_kafka_topic_partition_list_destroy(parts); /* Fire off any enqueued offset_commit_cb */ test_consumer_poll_no_msgs(what, rk, testid, 0); TEST_SAY("%s: committed_offset %"PRId64", expected_offset %"PRId64"\n", what, committed_offset, expected_offset); if (!auto_commit && !async) { /* Sync commits should be up to date at this point. */ if (committed_offset != expected_offset) TEST_FAIL("%s: Sync commit: committed offset %"PRId64 " should be same as expected offset " "%"PRId64, what, committed_offset, expected_offset); } else { /* Wait for offset commits to catch up */ while (committed_offset < expected_offset) { TEST_SAYL(3, "%s: Wait for committed offset %"PRId64 " to reach expected offset %"PRId64"\n", what, committed_offset, expected_offset); test_consumer_poll_no_msgs(what, rk, testid, 1000); } } TEST_SAY("%s: phase 1 complete, %d messages consumed, " "next expected offset is %"PRId64"\n", what, cnt, expected_offset); /* Issue #827: cause committed() to return prematurely by specifying * low timeout. The bug (use after free) will only * be catched by valgrind. */ do { parts = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(parts, topic, partition); err = rd_kafka_committed(rk, parts, 1); rd_kafka_topic_partition_list_destroy(parts); TEST_SAY("Issue #827: committed() returned %s\n", rd_kafka_err2str(err)); } while (err != RD_KAFKA_RESP_ERR__TIMED_OUT); /* Query position */ parts = rd_kafka_topic_partition_list_new(1); rd_kafka_topic_partition_list_add(parts, topic, partition); err = rd_kafka_committed(rk, parts, tmout_multip(5*1000)); if (err) TEST_FAIL("%s: committed() failed: %s", what, rd_kafka_err2str(err)); if (!(rktpar = rd_kafka_topic_partition_list_find(parts, topic, partition))) TEST_FAIL("%s: committed(): topic lost\n", what); if (rktpar->offset != expected_offset) TEST_FAIL("%s: Expected committed() offset %"PRId64", got %"PRId64, what, expected_offset, rktpar->offset); TEST_SAY("%s: Committed offset is at %"PRId64", good!\n", what, rktpar->offset); rd_kafka_topic_partition_list_destroy(parts); test_consumer_close(rk); rd_kafka_destroy(rk); /* Fire up a new consumer and continue from where we left off. */ TEST_SAY("%s: phase 2: starting new consumer to resume consumption\n",what); rk = test_create_handle(RD_KAFKA_CONSUMER, conf); rd_kafka_poll_set_consumer(rk); test_consumer_subscribe(rk, topic); while (cnt < msgcnt) { rd_kafka_message_t *rkm; rkm = rd_kafka_consumer_poll(rk, 10*1000); if (!rkm) continue; if (rkm->err == RD_KAFKA_RESP_ERR__TIMED_OUT) TEST_FAIL("%s: Timed out waiting for message %d", what,cnt); else if (rkm->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { rd_kafka_message_destroy(rkm); continue; } else if (rkm->err) TEST_FAIL("%s: Consumer error: %s", what, rd_kafka_message_errstr(rkm)); if (rkm->offset != expected_offset) TEST_FAIL("%s: Received message offset %"PRId64 ", expected %"PRId64" at msgcnt %d/%d\n", what, rkm->offset, expected_offset, cnt, msgcnt); rd_kafka_message_destroy(rkm); expected_offset++; cnt++; } TEST_SAY("%s: phase 2: complete\n", what); test_consumer_close(rk); rd_kafka_destroy(rk); TIMING_STOP(&t_all); }