static PyObject *Consumer_assignment (Handle *self, PyObject *args,
                                      PyObject *kwargs) {

        PyObject *plist;
        rd_kafka_topic_partition_list_t *c_parts;
        rd_kafka_resp_err_t err;

        if (!self->rk) {
                PyErr_SetString(PyExc_RuntimeError,
                                "Consumer closed");
                return NULL;
        }

        err = rd_kafka_assignment(self->rk, &c_parts);
        if (err) {
                cfl_PyErr_Format(err,
                                 "Failed to get assignment: %s",
                                 rd_kafka_err2str(err));
                return NULL;
        }


        plist = c_parts_to_py(c_parts);
        rd_kafka_topic_partition_list_destroy(c_parts);

        return plist;
}
Exemplo n.º 2
0
RdKafka::ErrorCode
RdKafka::KafkaConsumerImpl::assignment (std::vector<RdKafka::TopicPartition*> &partitions) {
  rd_kafka_topic_partition_list_t *c_parts;
  rd_kafka_resp_err_t err;

  if ((err = rd_kafka_assignment(rk_, &c_parts)))
    return static_cast<RdKafka::ErrorCode>(err);

  partitions.resize(c_parts->cnt);

  for (int i = 0 ; i < c_parts->cnt ; i++)
    partitions[i] = new RdKafka::TopicPartitionImpl(&c_parts->elems[i]);

  rd_kafka_topic_partition_list_destroy(c_parts);

  return RdKafka::ERR_NO_ERROR;
}
Exemplo n.º 3
0
int main_0093_holb_consumer (int argc, char **argv) {
        const char *topic = test_mk_topic_name("0093_holb_consumer", 1);
        int64_t testid;
        const int msgcnt = 100;
        struct _consumer c[_CONSUMER_CNT] = RD_ZERO_INIT;
        rd_kafka_conf_t *conf;

        testid = test_id_generate();

        test_conf_init(&conf, NULL, 60);

        test_create_topic(topic, 1, 1);

        test_produce_msgs_easy(topic, testid, 0, msgcnt);

        test_conf_set(conf, "session.timeout.ms", "6000");
        test_conf_set(conf, "max.poll.interval.ms", "20000");
        test_conf_set(conf, "socket.timeout.ms", "3000");
        test_conf_set(conf, "auto.offset.reset", "earliest");
        /* Trigger other requests often */
        test_conf_set(conf, "topic.metadata.refresh.interval.ms", "500");
        rd_kafka_conf_set_rebalance_cb(conf, rebalance_cb);

        rd_kafka_conf_set_opaque(conf, &c[0]);
        c[0].rk = test_create_consumer(topic, NULL,
                                       rd_kafka_conf_dup(conf), NULL);

        rd_kafka_conf_set_opaque(conf, &c[1]);
        c[1].rk = test_create_consumer(topic, NULL, conf, NULL);

        test_consumer_subscribe(c[0].rk, topic);

        /* c0: assign */
        c[0].max_rebalance_cnt = 1;

        /* c1: none, hasn't joined yet */
        c[1].max_rebalance_cnt = 0;

        TEST_SAY("Waiting for c[0] assignment\n");
        while (1) {
                rd_kafka_topic_partition_list_t *parts = NULL;

                do_consume(&c[0], 1/*1s*/);

                if (rd_kafka_assignment(c[0].rk, &parts) !=
                    RD_KAFKA_RESP_ERR_NO_ERROR ||
                    !parts || parts->cnt == 0) {
                        if (parts)
                                rd_kafka_topic_partition_list_destroy(parts);
                        continue;
                }

                TEST_SAY("%s got assignment of %d partition(s)\n",
                         rd_kafka_name(c[0].rk), parts->cnt);
                rd_kafka_topic_partition_list_destroy(parts);
                break;
        }

        TEST_SAY("c[0] got assignment, consuming..\n");
        do_consume(&c[0], 5/*5s*/);

        TEST_SAY("Joining second consumer\n");
        test_consumer_subscribe(c[1].rk, topic);

        /* Just poll second consumer for 10s, the rebalance will not
         * finish until the first consumer polls */
        do_consume(&c[1], 10/*10s*/);

        /* c0: the next call to do_consume/poll will trigger
         *     its rebalance callback, first revoke then assign. */
        c[0].max_rebalance_cnt += 2;
        /* c1: first rebalance */
        c[1].max_rebalance_cnt++;

        TEST_SAY("Expected rebalances: c[0]: %d/%d, c[1]: %d/%d\n",
                 c[0].rebalance_cnt, c[0].max_rebalance_cnt,
                 c[1].rebalance_cnt, c[1].max_rebalance_cnt);

        /* Let rebalances kick in, then consume messages. */
        while (c[0].cnt + c[1].cnt < msgcnt) {
                do_consume(&c[0], 0);
                do_consume(&c[1], 0);
        }

        /* Allow the extra revoke rebalance on close() */
        c[0].max_rebalance_cnt++;
        c[1].max_rebalance_cnt++;

        test_consumer_close(c[0].rk);
        test_consumer_close(c[1].rk);

        rd_kafka_destroy(c[0].rk);
        rd_kafka_destroy(c[1].rk);

        return 0;
}
Exemplo n.º 4
0
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);
}
Exemplo n.º 5
0
int main_0049_consume_conn_close (int argc, char **argv) {
        rd_kafka_t *rk;
        const char *topic = test_mk_topic_name("0049_consume_conn_close", 1);
        uint64_t testid;
        int msgcnt = test_on_ci ? 1000 : 10000;
        test_msgver_t mv;
        rd_kafka_conf_t *conf;
        rd_kafka_topic_conf_t *tconf;
        rd_kafka_topic_partition_list_t *assignment;
        rd_kafka_resp_err_t err;

        if (!test_conf_match(NULL, "sasl.mechanisms", "GSSAPI")) {
                TEST_SKIP("KNOWN ISSUE: ApiVersionRequest+SaslHandshake "
                          "will not play well with sudden disconnects\n");
                return 0;
        }

        test_conf_init(&conf, &tconf, 60);
        /* Want an even number so it is divisable by two without surprises */
        msgcnt = (msgcnt / (int)test_timeout_multiplier) & ~1;

        testid = test_id_generate();
        test_produce_msgs_easy(topic, testid, RD_KAFKA_PARTITION_UA, msgcnt);


        test_socket_enable(conf);
        test_curr->connect_cb = connect_cb;
        test_curr->is_fatal_cb = is_fatal_cb;

        test_topic_conf_set(tconf, "auto.offset.reset", "smallest");

        rk = test_create_consumer(topic, NULL, conf, tconf);

        test_consumer_subscribe(rk, topic);

        test_msgver_init(&mv, testid);

        test_consumer_poll("consume.up", rk, testid, -1, 0, msgcnt/2, &mv);

        err = rd_kafka_assignment(rk, &assignment);
        TEST_ASSERT(!err, "assignment() failed: %s", rd_kafka_err2str(err));
        TEST_ASSERT(assignment->cnt > 0, "empty assignment");

        TEST_SAY("Bringing down the network\n");

        TEST_LOCK();
        simulate_network_down = 1;
        TEST_UNLOCK();
        test_socket_close_all(test_curr, 1/*reinit*/);

        TEST_SAY("Waiting for session timeout to expire (6s), and then some\n");

        /* Commit an offset, which should fail, to trigger the offset commit
         * callback fallback (CONSUMER_ERR) */
        assignment->elems[0].offset = 123456789;
        TEST_SAY("Committing offsets while down, should fail eventually\n");
        err = rd_kafka_commit(rk, assignment, 1/*async*/);
        TEST_ASSERT(!err, "async commit failed: %s", rd_kafka_err2str(err));
        rd_kafka_topic_partition_list_destroy(assignment);

        rd_sleep(10);

        TEST_SAY("Bringing network back up\n");
        TEST_LOCK();
        simulate_network_down = 0;
        TEST_UNLOCK();

        TEST_SAY("Continuing to consume..\n");
        test_consumer_poll("consume.up2", rk, testid, -1, msgcnt/2, msgcnt/2,
                           &mv);

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

        test_msgver_clear(&mv);

        test_consumer_close(rk);
        rd_kafka_destroy(rk);

        return 0;
}
Exemplo n.º 6
0
int main_0006_symbols (int argc, char **argv) {

        if (argc < 0 /* always false */) {
                rd_kafka_version();
                rd_kafka_version_str();
		rd_kafka_get_debug_contexts();
		rd_kafka_get_err_descs(NULL, NULL);
                rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR);
		rd_kafka_err2name(RD_KAFKA_RESP_ERR_NO_ERROR);
                rd_kafka_errno2err(EINVAL);
                rd_kafka_errno();
		rd_kafka_last_error();
                rd_kafka_conf_new();
                rd_kafka_conf_destroy(NULL);
                rd_kafka_conf_dup(NULL);
                rd_kafka_conf_set(NULL, NULL, NULL, NULL, 0);
                rd_kafka_conf_set_dr_cb(NULL, NULL);
                rd_kafka_conf_set_dr_msg_cb(NULL, NULL);
                rd_kafka_conf_set_error_cb(NULL, NULL);
                rd_kafka_conf_set_stats_cb(NULL, NULL);
                rd_kafka_conf_set_log_cb(NULL, NULL);
                rd_kafka_conf_set_socket_cb(NULL, NULL);
		rd_kafka_conf_set_rebalance_cb(NULL, NULL);
		rd_kafka_conf_set_offset_commit_cb(NULL, NULL);
		rd_kafka_conf_set_throttle_cb(NULL, NULL);
		rd_kafka_conf_set_default_topic_conf(NULL, NULL);
		rd_kafka_conf_get(NULL, NULL, NULL, NULL);
#ifndef _MSC_VER
		rd_kafka_conf_set_open_cb(NULL, NULL);
#endif
		rd_kafka_conf_set_opaque(NULL, NULL);
                rd_kafka_opaque(NULL);
                rd_kafka_conf_dump(NULL, NULL);
                rd_kafka_topic_conf_dump(NULL, NULL);
                rd_kafka_conf_dump_free(NULL, 0);
                rd_kafka_conf_properties_show(NULL);
                rd_kafka_topic_conf_new();
                rd_kafka_topic_conf_dup(NULL);
                rd_kafka_topic_conf_destroy(NULL);
                rd_kafka_topic_conf_set(NULL, NULL, NULL, NULL, 0);
                rd_kafka_topic_conf_set_opaque(NULL, NULL);
		rd_kafka_topic_conf_get(NULL, NULL, NULL, NULL);
                rd_kafka_topic_conf_set_partitioner_cb(NULL, NULL);
                rd_kafka_topic_partition_available(NULL, 0);
		rd_kafka_topic_opaque(NULL);
                rd_kafka_msg_partitioner_random(NULL, NULL, 0, 0, NULL, NULL);
                rd_kafka_msg_partitioner_consistent(NULL, NULL, 0, 0, NULL, NULL);
                rd_kafka_msg_partitioner_consistent_random(NULL, NULL, 0, 0, NULL, NULL);
                rd_kafka_new(0, NULL, NULL, 0);
                rd_kafka_destroy(NULL);
                rd_kafka_name(NULL);
		rd_kafka_memberid(NULL);
                rd_kafka_topic_new(NULL, NULL, NULL);
                rd_kafka_topic_destroy(NULL);
                rd_kafka_topic_name(NULL);
                rd_kafka_message_destroy(NULL);
                rd_kafka_message_errstr(NULL);
		rd_kafka_message_timestamp(NULL, NULL);
                rd_kafka_consume_start(NULL, 0, 0);
                rd_kafka_consume_stop(NULL, 0);
                rd_kafka_consume(NULL, 0, 0);
                rd_kafka_consume_batch(NULL, 0, 0, NULL, 0);
                rd_kafka_consume_callback(NULL, 0, 0, NULL, NULL);
                rd_kafka_offset_store(NULL, 0, 0);
                rd_kafka_produce(NULL, 0, 0, NULL, 0, NULL, 0, NULL);
                rd_kafka_produce_batch(NULL, 0, 0, NULL, 0);
                rd_kafka_poll(NULL, 0);
                rd_kafka_brokers_add(NULL, NULL);
                /* DEPRECATED: rd_kafka_set_logger(NULL, NULL); */
                rd_kafka_set_log_level(NULL, 0);
                rd_kafka_log_print(NULL, 0, NULL, NULL);
#ifndef _MSC_VER
                rd_kafka_log_syslog(NULL, 0, NULL, NULL);
#endif
                rd_kafka_outq_len(NULL);
                rd_kafka_dump(NULL, NULL);
                rd_kafka_thread_cnt();
                rd_kafka_wait_destroyed(0);
                rd_kafka_metadata(NULL, 0, NULL, NULL, 0);
                rd_kafka_metadata_destroy(NULL);
                rd_kafka_queue_destroy(NULL);
                rd_kafka_consume_start_queue(NULL, 0, 0, NULL);
                rd_kafka_consume_queue(NULL, 0);
                rd_kafka_consume_batch_queue(NULL, 0, NULL, 0);
                rd_kafka_consume_callback_queue(NULL, 0, NULL, NULL);
                rd_kafka_seek(NULL, 0, 0, 0);
                rd_kafka_yield(NULL);
                rd_kafka_mem_free(NULL, NULL);
                rd_kafka_list_groups(NULL, NULL, NULL, 0);
                rd_kafka_group_list_destroy(NULL);

		/* KafkaConsumer API */
		rd_kafka_subscribe(NULL, NULL);
		rd_kafka_unsubscribe(NULL);
		rd_kafka_subscription(NULL, NULL);
		rd_kafka_consumer_poll(NULL, 0);
		rd_kafka_consumer_close(NULL);
		rd_kafka_assign(NULL, NULL);
		rd_kafka_assignment(NULL, NULL);
		rd_kafka_commit(NULL, NULL, 0);
		rd_kafka_commit_message(NULL, NULL, 0);
                rd_kafka_committed(NULL, NULL, 0);
		rd_kafka_position(NULL, NULL);

		/* TopicPartition */
		rd_kafka_topic_partition_list_new(0);
		rd_kafka_topic_partition_list_destroy(NULL);
		rd_kafka_topic_partition_list_add(NULL, NULL, 0);
		rd_kafka_topic_partition_list_add_range(NULL, NULL, 0, 0);
		rd_kafka_topic_partition_list_del(NULL, NULL, 0);
		rd_kafka_topic_partition_list_del_by_idx(NULL, 0);
		rd_kafka_topic_partition_list_copy(NULL);
		rd_kafka_topic_partition_list_set_offset(NULL, NULL, 0, 0);
		rd_kafka_topic_partition_list_find(NULL, NULL, 0);
		rd_kafka_query_watermark_offsets(NULL, NULL, 0, NULL, NULL, 0);
		rd_kafka_get_watermark_offsets(NULL, NULL, 0, NULL, NULL);
        }


	return 0;
}