static void do_test_fetch_max_bytes (void) { const int partcnt = 3; int msgcnt = 10 * partcnt; const int msgsize = 900*1024; /* Less than 1 Meg to account * for batch overhead */ std::string errstr; RdKafka::ErrorCode err; std::string topic = Test::mk_topic_name("0081-fetch_max_bytes", 1); /* Produce messages to partitions */ for (int32_t p = 0 ; p < (int32_t)partcnt ; p++) test_produce_msgs_easy_size(topic.c_str(), 0, p, msgcnt, msgsize); /* Create consumer */ RdKafka::Conf *conf; Test::conf_init(&conf, NULL, 10); Test::conf_set(conf, "group.id", topic); Test::conf_set(conf, "auto.offset.reset", "earliest"); /* We try to fetch 20 Megs per partition, but only allow 1 Meg as total * response size, this ends up serving the first batch from the * first partition. * receive.message.max.bytes is set low to trigger the original bug, * but this value is now adjusted upwards automatically by rd_kafka_new() * to hold both fetch.max.bytes and the protocol / batching overhead. * Prior to the introduction of fetch.max.bytes the fetcher code * would use receive.message.max.bytes to limit the total Fetch response, * but due to batching overhead it would result in situations where * the consumer asked for 1000000 bytes and got 1000096 bytes batch, which * was higher than the 1000000 limit. * See https://github.com/edenhill/librdkafka/issues/1616 * * With the added configuration strictness checks, a user-supplied * value is no longer over-written: * receive.message.max.bytes must be configured to be at least 512 bytes * larger than fetch.max.bytes. */ Test::conf_set(conf, "max.partition.fetch.bytes", "20000000"); /* ~20MB */ Test::conf_set(conf, "fetch.max.bytes", "1000000"); /* ~1MB */ Test::conf_set(conf, "receive.message.max.bytes", "1000512"); /* ~1MB+512 */ RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr); if (!c) Test::Fail("Failed to create KafkaConsumer: " + errstr); delete conf; /* Subscribe */ std::vector<std::string> topics; topics.push_back(topic); if ((err = c->subscribe(topics))) Test::Fail("subscribe failed: " + RdKafka::err2str(err)); /* Start consuming */ Test::Say("Consuming topic " + topic + "\n"); int cnt = 0; while (cnt < msgcnt) { RdKafka::Message *msg = c->consume(tmout_multip(1000)); switch (msg->err()) { case RdKafka::ERR__TIMED_OUT: break; case RdKafka::ERR_NO_ERROR: cnt++; break; default: Test::Fail("Consume error: " + msg->errstr()); break; } delete msg; } Test::Say("Done\n"); c->close(); delete c; }
int main (int argc, char **argv) { std::string brokers = "localhost"; std::string errstr; std::string topic_str; std::string mode; std::string debug; std::vector<std::string> topics; bool do_conf_dump = false; int opt; int use_ccb = 0; /* * Create configuration objects */ RdKafka::Conf *conf = RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL); RdKafka::Conf *tconf = RdKafka::Conf::create(RdKafka::Conf::CONF_TOPIC); ExampleRebalanceCb ex_rebalance_cb; conf->set("rebalance_cb", &ex_rebalance_cb, errstr); while ((opt = getopt(argc, argv, "g:b:z:qd:eX:AM:f:qv")) != -1) { switch (opt) { case 'g': if (conf->set("group.id", optarg, errstr) != RdKafka::Conf::CONF_OK) { std::cerr << errstr << std::endl; exit(1); } break; case 'b': brokers = optarg; break; case 'z': if (conf->set("compression.codec", optarg, errstr) != RdKafka::Conf::CONF_OK) { std::cerr << errstr << std::endl; exit(1); } break; case 'e': exit_eof = true; break; case 'd': debug = optarg; break; case 'M': if (conf->set("statistics.interval.ms", optarg, errstr) != RdKafka::Conf::CONF_OK) { std::cerr << errstr << std::endl; exit(1); } break; case 'X': { char *name, *val; if (!strcmp(optarg, "dump")) { do_conf_dump = true; continue; } name = optarg; if (!(val = strchr(name, '='))) { std::cerr << "%% Expected -X property=value, not " << name << std::endl; exit(1); } *val = '\0'; val++; /* Try "topic." prefixed properties on topic * conf first, and then fall through to global if * it didnt match a topic configuration property. */ RdKafka::Conf::ConfResult res = RdKafka::Conf::CONF_UNKNOWN; if (!strncmp(name, "topic.", strlen("topic."))) res = tconf->set(name+strlen("topic."), val, errstr); if (res == RdKafka::Conf::CONF_UNKNOWN) res = conf->set(name, val, errstr); if (res != RdKafka::Conf::CONF_OK) { std::cerr << errstr << std::endl; exit(1); } } break; case 'f': if (!strcmp(optarg, "ccb")) use_ccb = 1; else { std::cerr << "Unknown option: " << optarg << std::endl; exit(1); } break; case 'q': verbosity--; break; case 'v': verbosity++; break; default: goto usage; } } for (; optind < argc ; optind++) topics.push_back(std::string(argv[optind])); if (topics.empty() || optind != argc) { usage: fprintf(stderr, "Usage: %s -g <group-id> [options] topic1 topic2..\n" "\n" "librdkafka version %s (0x%08x)\n" "\n" " Options:\n" " -g <group-id> Consumer group id\n" " -b <brokers> Broker address (localhost:9092)\n" " -z <codec> Enable compression:\n" " none|gzip|snappy\n" " -e Exit consumer when last message\n" " in partition has been received.\n" " -d [facs..] Enable debugging contexts:\n" " %s\n" " -M <intervalms> Enable statistics\n" " -X <prop=name> Set arbitrary librdkafka " "configuration property\n" " Properties prefixed with \"topic.\" " "will be set on topic object.\n" " Use '-X list' to see the full list\n" " of supported properties.\n" " -f <flag> Set option:\n" " ccb - use consume_callback\n" " -q Quiet / Decrease verbosity\n" " -v Increase verbosity\n" "\n" "\n", argv[0], RdKafka::version_str().c_str(), RdKafka::version(), RdKafka::get_debug_contexts().c_str()); exit(1); } /* * Set configuration properties */ conf->set("metadata.broker.list", brokers, errstr); if (!debug.empty()) { if (conf->set("debug", debug, errstr) != RdKafka::Conf::CONF_OK) { std::cerr << errstr << std::endl; exit(1); } } ExampleEventCb ex_event_cb; conf->set("event_cb", &ex_event_cb, errstr); conf->set("default_topic_conf", tconf, errstr); delete tconf; if (do_conf_dump) { int pass; for (pass = 0 ; pass < 2 ; pass++) { std::list<std::string> *dump; if (pass == 0) { dump = conf->dump(); std::cout << "# Global config" << std::endl; } else { dump = tconf->dump(); std::cout << "# Topic config" << std::endl; } for (std::list<std::string>::iterator it = dump->begin(); it != dump->end(); ) { std::cout << *it << " = "; it++; std::cout << *it << std::endl; it++; } std::cout << std::endl; } exit(0); } signal(SIGINT, sigterm); signal(SIGTERM, sigterm); /* * Consumer mode */ /* * Create consumer using accumulated global configuration. */ RdKafka::KafkaConsumer *consumer = RdKafka::KafkaConsumer::create(conf, errstr); if (!consumer) { std::cerr << "Failed to create consumer: " << errstr << std::endl; exit(1); } delete conf; std::cout << "% Created consumer " << consumer->name() << std::endl; /* * Subscribe to topics */ RdKafka::ErrorCode err = consumer->subscribe(topics); if (err) { std::cerr << "Failed to subscribe to " << topics.size() << " topics: " << RdKafka::err2str(err) << std::endl; exit(1); } /* * Consume messages */ while (run) { if (use_ccb) { std::cerr << "Use callback: Not implemented" << std::endl; break; } RdKafka::Message *msg = consumer->consume(1000); msg_consume(msg, NULL); delete msg; } alarm(10); /* * Stop consumer */ consumer->close(); delete consumer; std::cerr << "% Consumed " << msg_cnt << " messages (" << msg_bytes << " bytes)" << std::endl; /* * Wait for RdKafka to decommission. * This is not strictly needed (with check outq_len() above), but * allows RdKafka to clean up all its resources before the application * exits so that memory profilers such as valgrind wont complain about * memory leaks. */ RdKafka::wait_destroyed(5000); return 0; }