|
Packit |
2997f0 |
/*
|
|
Packit |
2997f0 |
* librdkafka - Apache Kafka C library
|
|
Packit |
2997f0 |
*
|
|
Packit |
2997f0 |
* Copyright (c) 2016, Magnus Edenhill
|
|
Packit |
2997f0 |
* All rights reserved.
|
|
Packit |
2997f0 |
*
|
|
Packit |
2997f0 |
* Redistribution and use in source and binary forms, with or without
|
|
Packit |
2997f0 |
* modification, are permitted provided that the following conditions are met:
|
|
Packit |
2997f0 |
*
|
|
Packit |
2997f0 |
* 1. Redistributions of source code must retain the above copyright notice,
|
|
Packit |
2997f0 |
* this list of conditions and the following disclaimer.
|
|
Packit |
2997f0 |
* 2. Redistributions in binary form must reproduce the above copyright notice,
|
|
Packit |
2997f0 |
* this list of conditions and the following disclaimer in the documentation
|
|
Packit |
2997f0 |
* and/or other materials provided with the distribution.
|
|
Packit |
2997f0 |
*
|
|
Packit |
2997f0 |
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
|
|
Packit |
2997f0 |
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
|
|
Packit |
2997f0 |
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
|
|
Packit |
2997f0 |
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
|
|
Packit |
2997f0 |
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
|
|
Packit |
2997f0 |
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
|
|
Packit |
2997f0 |
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
|
|
Packit |
2997f0 |
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
|
|
Packit |
2997f0 |
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
|
|
Packit |
2997f0 |
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
|
|
Packit |
2997f0 |
* POSSIBILITY OF SUCH DAMAGE.
|
|
Packit |
2997f0 |
*/
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
#include <iostream>
|
|
Packit |
2997f0 |
#include <cstring>
|
|
Packit |
2997f0 |
#include <cstdlib>
|
|
Packit |
2997f0 |
#include "testcpp.h"
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/**
|
|
Packit |
2997f0 |
* Verify consumer_lag
|
|
Packit |
2997f0 |
*/
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
static std::string topic;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
class StatsCb : public RdKafka::EventCb {
|
|
Packit |
2997f0 |
public:
|
|
Packit |
2997f0 |
int64_t calc_lag; //calculated lag
|
|
Packit |
2997f0 |
int lag_valid; // number of times lag has been valid
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
StatsCb() {
|
|
Packit |
2997f0 |
calc_lag = -1;
|
|
Packit |
2997f0 |
lag_valid = 0;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/**
|
|
Packit |
2997f0 |
* @brief Event callback
|
|
Packit |
2997f0 |
*/
|
|
Packit |
2997f0 |
void event_cb (RdKafka::Event &event) {
|
|
Packit |
2997f0 |
if (event.type() == RdKafka::Event::EVENT_LOG) {
|
|
Packit |
2997f0 |
Test::Say(tostr() << "LOG-" << event.severity() << "-" << event.fac() <<
|
|
Packit |
2997f0 |
": " << event.str() << "\n");
|
|
Packit |
2997f0 |
return;
|
|
Packit |
2997f0 |
} else if (event.type() != RdKafka::Event::EVENT_STATS) {
|
|
Packit |
2997f0 |
Test::Say(tostr() << "Dropping event " << event.type() << "\n");
|
|
Packit |
2997f0 |
return;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
int64_t consumer_lag = parse_json(event.str().c_str());
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
Test::Say(3, tostr() << "Stats: consumer_lag is " << consumer_lag << "\n");
|
|
Packit |
2997f0 |
if (consumer_lag == -1) {
|
|
Packit |
2997f0 |
Test::Say(2, "Skipping old stats with invalid consumer_lag\n");
|
|
Packit |
2997f0 |
return; /* Old stats generated before first message consumed */
|
|
Packit |
2997f0 |
} else if (consumer_lag != calc_lag)
|
|
Packit |
2997f0 |
Test::Fail(tostr() << "Stats consumer_lag " << consumer_lag << ", expected " << calc_lag << "\n");
|
|
Packit |
2997f0 |
else
|
|
Packit |
2997f0 |
lag_valid++;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/**
|
|
Packit |
2997f0 |
* @brief Naiive JSON parsing, find the consumer_lag for partition 0
|
|
Packit |
2997f0 |
* and return it.
|
|
Packit |
2997f0 |
*/
|
|
Packit |
2997f0 |
static int64_t parse_json (const char *json_doc) {
|
|
Packit |
2997f0 |
const std::string match_topic(std::string("\"") + topic + "\":");
|
|
Packit |
2997f0 |
const char *search[] = { "\"topics\":",
|
|
Packit |
2997f0 |
match_topic.c_str(),
|
|
Packit |
2997f0 |
"\"partitions\":",
|
|
Packit |
2997f0 |
"\"0\":",
|
|
Packit |
2997f0 |
"\"consumer_lag\":",
|
|
Packit |
2997f0 |
NULL };
|
|
Packit |
2997f0 |
const char *remain = json_doc;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
for (const char **sp = search ; *sp ; sp++) {
|
|
Packit |
2997f0 |
const char *t = strstr(remain, *sp);
|
|
Packit |
2997f0 |
if (!t)
|
|
Packit |
2997f0 |
Test::Fail(tostr() << "Couldnt find " << *sp <<
|
|
Packit |
2997f0 |
" in remaining stats output:\n" << remain <<
|
|
Packit |
2997f0 |
"\n====================\n" << json_doc << "\n");
|
|
Packit |
2997f0 |
remain = t + strlen(*sp);
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
while (*remain == ' ')
|
|
Packit |
2997f0 |
remain++;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
if (!*remain)
|
|
Packit |
2997f0 |
Test::Fail("Nothing following consumer_lag");
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
int64_t lag = strtoull(remain, NULL, 0);
|
|
Packit |
2997f0 |
if (lag == -1)
|
|
Packit |
2997f0 |
Test::Say(tostr() << "Consumer lag " << lag << " is invalid, stats:\n" <<
|
|
Packit |
2997f0 |
json_doc << "\n");
|
|
Packit |
2997f0 |
return lag;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
};
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
static void do_test_consumer_lag (void) {
|
|
Packit |
2997f0 |
const int msgcnt = 10;
|
|
Packit |
2997f0 |
std::string errstr;
|
|
Packit |
2997f0 |
RdKafka::ErrorCode err;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
topic = Test::mk_topic_name("0061-consumer_lag", 1);
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
test_produce_msgs_easy(topic.c_str(), 0, 0, msgcnt);
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/*
|
|
Packit |
2997f0 |
* Create consumer
|
|
Packit |
2997f0 |
*/
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/* Create consumer */
|
|
Packit |
2997f0 |
RdKafka::Conf *conf;
|
|
Packit |
2997f0 |
Test::conf_init(&conf, NULL, 10);
|
|
Packit |
2997f0 |
StatsCb stats;
|
|
Packit |
2997f0 |
if (conf->set("event_cb", &stats, errstr) != RdKafka::Conf::CONF_OK)
|
|
Packit |
2997f0 |
Test::Fail("set event_cb failed: " + errstr);
|
|
Packit |
2997f0 |
Test::conf_set(conf, "group.id", topic);
|
|
Packit |
2997f0 |
Test::conf_set(conf, "enable.auto.commit", "false");
|
|
Packit |
2997f0 |
Test::conf_set(conf, "enable.partition.eof", "false");
|
|
Packit |
2997f0 |
Test::conf_set(conf, "auto.offset.reset", "earliest");
|
|
Packit |
2997f0 |
Test::conf_set(conf, "statistics.interval.ms", "100");
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
RdKafka::KafkaConsumer *c = RdKafka::KafkaConsumer::create(conf, errstr);
|
|
Packit |
2997f0 |
if (!c)
|
|
Packit |
2997f0 |
Test::Fail("Failed to create KafkaConsumer: " + errstr);
|
|
Packit |
2997f0 |
delete conf;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/* Assign partitions */
|
|
Packit |
2997f0 |
/* Subscribe */
|
|
Packit |
2997f0 |
std::vector<RdKafka::TopicPartition*> parts;
|
|
Packit |
2997f0 |
parts.push_back(RdKafka::TopicPartition::create(topic, 0));
|
|
Packit |
2997f0 |
if ((err = c->assign(parts)))
|
|
Packit |
2997f0 |
Test::Fail("assign failed: " + RdKafka::err2str(err));
|
|
Packit |
2997f0 |
RdKafka::TopicPartition::destroy(parts);
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
/* Start consuming */
|
|
Packit |
2997f0 |
Test::Say("Consuming topic " + topic + "\n");
|
|
Packit |
2997f0 |
int cnt = 0;
|
|
Packit |
2997f0 |
while (cnt < msgcnt) {
|
|
Packit |
2997f0 |
RdKafka::Message *msg = c->consume(tmout_multip(1000));
|
|
Packit |
2997f0 |
switch (msg->err())
|
|
Packit |
2997f0 |
{
|
|
Packit |
2997f0 |
case RdKafka::ERR__TIMED_OUT:
|
|
Packit |
2997f0 |
break;
|
|
Packit |
2997f0 |
case RdKafka::ERR__PARTITION_EOF:
|
|
Packit |
2997f0 |
Test::Fail(tostr() << "Consume error after " << cnt << "/" << msgcnt << " messages: " << msg->errstr());
|
|
Packit |
2997f0 |
break;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
case RdKafka::ERR_NO_ERROR:
|
|
Packit |
2997f0 |
/* Proper message. Updated calculated lag for later
|
|
Packit |
2997f0 |
* checking in stats callback */
|
|
Packit |
2997f0 |
stats.calc_lag = msgcnt - (msg->offset()+1);
|
|
Packit |
2997f0 |
cnt++;
|
|
Packit |
2997f0 |
Test::Say(2, tostr() << "Received message #" << cnt << "/" << msgcnt <<
|
|
Packit |
2997f0 |
" at offset " << msg->offset() << " (calc lag " << stats.calc_lag << ")\n");
|
|
Packit |
2997f0 |
/* Slow down message "processing" to make sure we get
|
|
Packit |
2997f0 |
* at least one stats callback per message. */
|
|
Packit |
2997f0 |
if (cnt < msgcnt)
|
|
Packit |
2997f0 |
rd_sleep(1);
|
|
Packit |
2997f0 |
break;
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
default:
|
|
Packit |
2997f0 |
Test::Fail("Consume error: " + msg->errstr());
|
|
Packit |
2997f0 |
break;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
delete msg;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
Test::Say(tostr() << "Done, lag was valid " <<
|
|
Packit |
2997f0 |
stats.lag_valid << " times\n");
|
|
Packit |
2997f0 |
if (stats.lag_valid == 0)
|
|
Packit |
2997f0 |
Test::Fail("No valid consumer_lag in statistics seen");
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
c->close();
|
|
Packit |
2997f0 |
delete c;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
|
|
Packit |
2997f0 |
extern "C" {
|
|
Packit |
2997f0 |
int main_0061_consumer_lag (int argc, char **argv) {
|
|
Packit |
2997f0 |
do_test_consumer_lag();
|
|
Packit |
2997f0 |
return 0;
|
|
Packit |
2997f0 |
}
|
|
Packit |
2997f0 |
}
|