[Bug][RoutineLoad] Avoid TOO_MANY_TASKS error (#6342)

Use `commitAsync` to commit offset to kafka, instead of using `commitSync`, which may block for a long time.
Also assign a group.id to routine load if user not specified "property.group.id" property, so that all consumer of
this job will use same group.id instead of a random id for each consume task.
This commit is contained in:
Mingyu Chen
2021-08-03 11:59:06 +08:00
committed by GitHub
parent 748604ff4f
commit 2c208e932b
6 changed files with 72 additions and 46 deletions

View File

@ -33,6 +33,7 @@
namespace doris {
static const std::string PROP_GROUP_ID = "group.id";
// init kafka consumer will only set common configs such as
// brokers, groupid
Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
@ -47,10 +48,6 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
// conf has to be deleted finally
Defer delete_conf{[conf]() { delete conf; }};
std::stringstream ss;
ss << BackendOptions::get_localhost() << "_";
std::string group_id = ss.str() + UniqueId::gen_uid().to_string();
LOG(INFO) << "init kafka consumer with group id: " << group_id;
std::string errstr;
auto set_conf = [&conf, &errstr](const std::string& conf_key, const std::string& conf_val) {
@ -74,7 +71,6 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
};
RETURN_IF_ERROR(set_conf("metadata.broker.list", ctx->kafka_info->brokers));
RETURN_IF_ERROR(set_conf("group.id", group_id));
RETURN_IF_ERROR(set_conf("enable.partition.eof", "false"));
RETURN_IF_ERROR(set_conf("enable.auto.offset.store", "false"));
// TODO: set it larger than 0 after we set rd_kafka_conf_set_stats_cb()
@ -108,6 +104,18 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
_custom_properties.emplace(item.first, item.second);
}
// if not specified group id, generate a random one.
// ATTN: In the new version, we have set a group.id on the FE side for jobs that have not set a groupid,
// but in order to ensure compatibility, we still do a check here.
if (_custom_properties.find(PROP_GROUP_ID) == _custom_properties.end()) {
std::stringstream ss;
ss << BackendOptions::get_localhost() << "_";
std::string group_id = ss.str() + UniqueId::gen_uid().to_string();
RETURN_IF_ERROR(set_conf(PROP_GROUP_ID, group_id));
_custom_properties.emplace(PROP_GROUP_ID, group_id);
}
LOG(INFO) << "init kafka consumer with group id: " << _custom_properties[PROP_GROUP_ID];
if (conf->set("event_cb", &_k_event_cb, errstr) != RdKafka::Conf::CONF_OK) {
std::stringstream ss;
ss << "PAUSE: failed to set 'event_cb'";
@ -354,7 +362,9 @@ Status KafkaDataConsumer::reset() {
}
Status KafkaDataConsumer::commit(std::vector<RdKafka::TopicPartition*>& offset) {
RdKafka::ErrorCode err = _k_consumer->commitSync(offset);
// Use async commit so that it will not block for a long time.
// Commit failure has no effect on Doris, subsequent tasks will continue to commit the new offset
RdKafka::ErrorCode err = _k_consumer->commitAsync(offset);
if (err != RdKafka::ERR_NO_ERROR) {
std::stringstream ss;
ss << "failed to commit kafka offset : " << RdKafka::err2str(err);