forked from StarRocks/starrocks
-
Notifications
You must be signed in to change notification settings - Fork 1
/
data_consumer.cpp
382 lines (338 loc) · 14.9 KB
/
data_consumer.cpp
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
// This file is made available under Elastic License 2.0.
// This file is based on code available under the Apache license here:
// https://github.com/apache/incubator-doris/blob/master/be/src/runtime/routine_load/data_consumer.cpp
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "runtime/routine_load/data_consumer.h"
#include <algorithm>
#include <functional>
#include <string>
#include <vector>
#include "common/status.h"
#include "gutil/strings/split.h"
#include "runtime/small_file_mgr.h"
#include "service/backend_options.h"
#include "util/defer_op.h"
#include "util/stopwatch.hpp"
#include "util/uid_util.h"
namespace starrocks {
// init kafka consumer will only set common configs such as
// brokers, groupid
Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
std::unique_lock<std::mutex> l(_lock);
if (_init) {
// this consumer has already been initialized.
return Status::OK();
}
RdKafka::Conf* conf = RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL);
// conf has to be deleted finally
auto conf_deleter = [conf]() { delete conf; };
DeferOp delete_conf([conf_deleter] { return conf_deleter(); });
std::string group_id;
auto it = ctx->kafka_info->properties.find("group.id");
if (it == ctx->kafka_info->properties.end()) {
group_id = BackendOptions::get_localhost() + "_" + UniqueId::gen_uid().to_string();
} else {
group_id = it->second;
}
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) {
RdKafka::Conf::ConfResult res = conf->set(conf_key, conf_val, errstr);
if (res == RdKafka::Conf::CONF_UNKNOWN) {
// ignore unknown config
return Status::OK();
} else if (errstr.find("not supported") != std::string::npos) {
// some java-only properties may be passed to here, and librdkafak will return 'xxx' not supported
// ignore it
return Status::OK();
} else if (res != RdKafka::Conf::CONF_OK) {
std::stringstream ss;
ss << "PAUSE: failed to set '" << conf_key << "', value: '" << conf_val << "', err: " << errstr;
LOG(WARNING) << ss.str();
return Status::InternalError(ss.str());
}
VLOG(3) << "set " << conf_key << ": " << conf_val;
return Status::OK();
};
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()
RETURN_IF_ERROR(set_conf("statistics.interval.ms", "0"));
RETURN_IF_ERROR(set_conf("auto.offset.reset", "error"));
RETURN_IF_ERROR(set_conf("api.version.request", "true"));
RETURN_IF_ERROR(set_conf("api.version.fallback.ms", "0"));
if (config::dependency_librdkafka_debug_enable) {
RETURN_IF_ERROR(set_conf("debug", config::dependency_librdkafka_debug));
}
for (auto& item : ctx->kafka_info->properties) {
if (boost::algorithm::starts_with(item.second, "FILE:")) {
// file property should has format: FILE:file_id:md5
std::vector<std::string> parts = strings::Split(item.second, ":", strings::SkipWhitespace());
if (parts.size() != 3) {
return Status::InternalError("PAUSE: Invalid file property of kafka: " + item.second);
}
int64_t file_id = std::stol(parts[1]);
std::string file_path;
Status st = ctx->exec_env()->small_file_mgr()->get_file(file_id, parts[2], &file_path);
if (!st.ok()) {
std::stringstream ss;
ss << "PAUSE: failed to get file for config: " << item.first << ", error: " << st.get_error_msg();
return Status::InternalError(ss.str());
}
RETURN_IF_ERROR(set_conf(item.first, file_path));
} else {
RETURN_IF_ERROR(set_conf(item.first, item.second));
}
_custom_properties.emplace(item.first, item.second);
}
if (conf->set("event_cb", &_k_event_cb, errstr) != RdKafka::Conf::CONF_OK) {
std::stringstream ss;
ss << "PAUSE: failed to set 'event_cb'";
LOG(WARNING) << ss.str();
return Status::InternalError(ss.str());
}
// create consumer
_k_consumer = RdKafka::KafkaConsumer::create(conf, errstr);
if (!_k_consumer) {
LOG(WARNING) << "PAUSE: failed to create kafka consumer: " << errstr;
return Status::InternalError("PAUSE: failed to create kafka consumer: " + errstr);
}
VLOG(3) << "finished to init kafka consumer. " << ctx->brief();
_init = true;
return Status::OK();
}
Status KafkaDataConsumer::assign_topic_partitions(const std::map<int32_t, int64_t>& begin_partition_offset,
const std::string& topic, StreamLoadContext* ctx) {
DCHECK(_k_consumer);
// create TopicPartitions
std::stringstream ss;
std::vector<RdKafka::TopicPartition*> topic_partitions;
for (auto& entry : begin_partition_offset) {
RdKafka::TopicPartition* tp1 = RdKafka::TopicPartition::create(topic, entry.first, entry.second);
topic_partitions.push_back(tp1);
ss << "[" << entry.first << ": " << entry.second << "] ";
}
LOG(INFO) << "consumer: " << _id << ", grp: " << _grp_id << " assign topic partitions: " << topic << ", "
<< ss.str();
// delete TopicPartition finally
auto tp_deleter = [&topic_partitions]() {
std::for_each(topic_partitions.begin(), topic_partitions.end(),
[](RdKafka::TopicPartition* tp1) { delete tp1; });
};
DeferOp delete_tp([tp_deleter] { return tp_deleter(); });
// assign partition
RdKafka::ErrorCode err = _k_consumer->assign(topic_partitions);
if (err) {
LOG(WARNING) << "failed to assign topic partitions: " << ctx->brief(true) << ", err: " << RdKafka::err2str(err);
return Status::InternalError("failed to assign topic partitions");
}
return Status::OK();
}
Status KafkaDataConsumer::group_consume(TimedBlockingQueue<RdKafka::Message*>* queue, int64_t max_running_time_ms) {
_last_visit_time = time(nullptr);
int64_t left_time = max_running_time_ms;
LOG(INFO) << "start kafka consumer: " << _id << ", grp: " << _grp_id << ", max running time(ms): " << left_time;
int64_t received_rows = 0;
int64_t put_rows = 0;
Status st = Status::OK();
MonotonicStopWatch consumer_watch;
MonotonicStopWatch watch;
watch.start();
while (true) {
{
std::unique_lock<std::mutex> l(_lock);
if (_cancelled) {
break;
}
}
if (left_time <= 0) {
break;
}
bool done = false;
// consume 1 message at a time
consumer_watch.start();
int64_t consume_timeout = std::min<int64_t>(left_time, config::routine_load_kafka_timeout_second * 1000);
std::unique_ptr<RdKafka::Message> msg(_k_consumer->consume(consume_timeout /* timeout, ms */));
consumer_watch.stop();
switch (msg->err()) {
case RdKafka::ERR_NO_ERROR:
if (!queue->blocking_put(msg.get())) {
// queue is shutdown
done = true;
} else {
++put_rows;
msg.release(); // release the ownership, msg will be deleted after being processed
}
++received_rows;
break;
case RdKafka::ERR__TIMED_OUT: {
// leave the status as OK, because this may happened
// if there is no data in kafka.
std::stringstream ss;
ss << msg->errstr() << ", timeout " << consume_timeout;
LOG(INFO) << "kafka consume timeout: " << _id << " msg " << ss.str();
break;
}
case RdKafka::ERR_OFFSET_OUT_OF_RANGE: {
done = true;
std::stringstream ss;
ss << msg->errstr() << ", partition " << msg->partition() << " offset " << msg->offset() << " has no data";
LOG(WARNING) << "kafka consume failed: " << _id << ", msg: " << ss.str();
st = Status::InternalError(ss.str());
break;
}
default:
LOG(WARNING) << "kafka consume failed: " << _id << ", msg: " << msg->errstr();
done = true;
st = Status::InternalError(msg->errstr());
break;
}
left_time = max_running_time_ms - watch.elapsed_time() / 1000 / 1000;
if (done) {
break;
}
}
LOG(INFO) << "kafka consume done: " << _id << ", grp: " << _grp_id << ". cancelled: " << _cancelled
<< ", left time(ms): " << left_time << ", total cost(ms): " << watch.elapsed_time() / 1000 / 1000
<< ", consume cost(ms): " << consumer_watch.elapsed_time() / 1000 / 1000
<< ", received rows: " << received_rows << ", put rows: " << put_rows;
return st;
}
Status KafkaDataConsumer::get_partition_offset(std::vector<int32_t>* partition_ids,
std::vector<int64_t>* beginning_offsets,
std::vector<int64_t>* latest_offsets) {
_last_visit_time = time(nullptr);
beginning_offsets->reserve(partition_ids->size());
latest_offsets->reserve(partition_ids->size());
for (auto p_id : *partition_ids) {
int64_t beginning_offset;
int64_t latest_offset;
RdKafka::ErrorCode err = _k_consumer->query_watermark_offsets(_topic, p_id, &beginning_offset, &latest_offset,
config::routine_load_kafka_timeout_second * 1000);
if (err != RdKafka::ERR_NO_ERROR) {
LOG(WARNING) << "failed to query watermark offset of topic: " << _topic << " partition: " << p_id
<< ", err: " << RdKafka::err2str(err);
return Status::InternalError("failed to query watermark offset, err: " + RdKafka::err2str(err));
}
beginning_offsets->push_back(beginning_offset);
latest_offsets->push_back(latest_offset);
}
return Status::OK();
}
Status KafkaDataConsumer::get_partition_meta(std::vector<int32_t>* partition_ids) {
_last_visit_time = time(nullptr);
// create topic conf
RdKafka::Conf* tconf = RdKafka::Conf::create(RdKafka::Conf::CONF_TOPIC);
auto conf_deleter = [tconf]() { delete tconf; };
DeferOp delete_conf([conf_deleter] { return conf_deleter(); });
// create topic
std::string errstr;
RdKafka::Topic* topic = RdKafka::Topic::create(_k_consumer, _topic, tconf, errstr);
if (topic == nullptr) {
std::stringstream ss;
ss << "failed to create topic: " << errstr;
LOG(WARNING) << ss.str();
return Status::InternalError(ss.str());
}
auto topic_deleter = [topic]() { delete topic; };
DeferOp delete_topic([topic_deleter] { return topic_deleter(); });
// get topic metadata
RdKafka::Metadata* metadata = nullptr;
RdKafka::ErrorCode err = _k_consumer->metadata(true /* for this topic */, topic, &metadata,
config::routine_load_kafka_timeout_second * 1000);
if (err != RdKafka::ERR_NO_ERROR) {
std::stringstream ss;
ss << "failed to get partition meta: " << RdKafka::err2str(err);
LOG(WARNING) << ss.str();
return Status::InternalError(ss.str());
}
auto meta_deleter = [metadata]() { delete metadata; };
DeferOp delete_meta([meta_deleter] { return meta_deleter(); });
// get partition ids
RdKafka::Metadata::TopicMetadataIterator it;
for (it = metadata->topics()->begin(); it != metadata->topics()->end(); ++it) {
if ((*it)->topic() != _topic) {
continue;
}
if ((*it)->err() != RdKafka::ERR_NO_ERROR) {
std::stringstream ss;
ss << "error: " << err2str((*it)->err());
if ((*it)->err() == RdKafka::ERR_LEADER_NOT_AVAILABLE) {
ss << ", try again";
}
LOG(WARNING) << ss.str();
return Status::InternalError(ss.str());
}
RdKafka::TopicMetadata::PartitionMetadataIterator ip;
for (ip = (*it)->partitions()->begin(); ip != (*it)->partitions()->end(); ++ip) {
partition_ids->push_back((*ip)->id());
}
}
if (partition_ids->empty()) {
return Status::InternalError("no partition in this topic");
}
return Status::OK();
}
Status KafkaDataConsumer::cancel(StreamLoadContext* ctx) {
std::unique_lock<std::mutex> l(_lock);
if (!_init) {
return Status::InternalError("consumer is not initialized");
}
_cancelled = true;
LOG(INFO) << "kafka consumer cancelled. " << _id;
return Status::OK();
}
Status KafkaDataConsumer::reset() {
std::unique_lock<std::mutex> l(_lock);
_cancelled = false;
_k_consumer->unassign();
return Status::OK();
}
Status KafkaDataConsumer::commit(std::vector<RdKafka::TopicPartition*>& offset) {
RdKafka::ErrorCode err = _k_consumer->commitSync(offset);
if (err != RdKafka::ERR_NO_ERROR) {
std::stringstream ss;
ss << "failed to commit kafka offset : " << RdKafka::err2str(err);
return Status::InternalError(ss.str());
}
return Status::OK();
}
// if the kafka brokers and topic are same,
// we considered this consumer as matched, thus can be reused.
bool KafkaDataConsumer::match(StreamLoadContext* ctx) {
if (ctx->load_src_type != TLoadSourceType::KAFKA) {
return false;
}
if (_brokers != ctx->kafka_info->brokers || _topic != ctx->kafka_info->topic) {
return false;
}
// check properties
if (_custom_properties.size() != ctx->kafka_info->properties.size()) {
return false;
}
for (auto& item : ctx->kafka_info->properties) {
auto it = _custom_properties.find(item.first);
if (it == _custom_properties.end() || it->second != item.second) {
return false;
}
}
return true;
}
} // end namespace starrocks