-
Notifications
You must be signed in to change notification settings - Fork 215
/
Copy pathkafka_handle_base.cpp
319 lines (269 loc) · 11.3 KB
/
kafka_handle_base.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
/*
* Copyright (c) 2017, Matias Fontanini
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met:
*
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above
* copyright notice, this list of conditions and the following disclaimer
* in the documentation and/or other materials provided with the
* distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
*/
#include "kafka_handle_base.h"
#include "metadata.h"
#include "group_information.h"
#include "exceptions.h"
#include "topic.h"
#include "topic_partition_list.h"
using std::string;
using std::vector;
using std::move;
using std::make_tuple;
using std::lock_guard;
using std::mutex;
using std::exception;
using std::chrono::milliseconds;
namespace cppkafka {
const milliseconds KafkaHandleBase::DEFAULT_TIMEOUT{1000};
KafkaHandleBase::KafkaHandleBase(Configuration config)
: timeout_ms_(DEFAULT_TIMEOUT), config_(move(config)), handle_(nullptr, HandleDeleter(this)), destroy_flags_(0) {
auto& maybe_config = config_.get_default_topic_configuration();
if (maybe_config) {
maybe_config->set_as_opaque();
auto conf_handle = rd_kafka_topic_conf_dup(maybe_config->get_handle());
rd_kafka_conf_set_default_topic_conf(config_.get_handle(), conf_handle);
}
}
void KafkaHandleBase::pause_partitions(const TopicPartitionList& topic_partitions) {
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_pause_partitions(get_handle(),
topic_list_handle.get());
check_error(error, topic_list_handle.get());
}
void KafkaHandleBase::pause(const std::string& topic) {
pause_partitions(convert(topic, get_metadata(get_topic(topic)).get_partitions()));
}
void KafkaHandleBase::resume_partitions(const TopicPartitionList& topic_partitions) {
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_resume_partitions(get_handle(),
topic_list_handle.get());
check_error(error, topic_list_handle.get());
}
void KafkaHandleBase::resume(const std::string& topic) {
resume_partitions(convert(topic, get_metadata(get_topic(topic)).get_partitions()));
}
void KafkaHandleBase::set_timeout(milliseconds timeout) {
timeout_ms_ = timeout;
}
void KafkaHandleBase::set_log_level(LogLevel level) {
rd_kafka_set_log_level(handle_.get(), static_cast<int>(level));
}
void KafkaHandleBase::add_brokers(const string& brokers) {
rd_kafka_brokers_add(handle_.get(), brokers.data());
}
rd_kafka_t* KafkaHandleBase::get_handle() const {
return handle_.get();
}
Topic KafkaHandleBase::get_topic(const string& name) {
save_topic_config(name, TopicConfiguration{});
return get_topic(name, nullptr);
}
Topic KafkaHandleBase::get_topic(const string& name, TopicConfiguration config) {
auto handle = config.get_handle();
save_topic_config(name, move(config));
return get_topic(name, rd_kafka_topic_conf_dup(handle));
}
KafkaHandleBase::OffsetTuple
KafkaHandleBase::query_offsets(const TopicPartition& topic_partition) const {
return query_offsets(topic_partition, timeout_ms_);
}
KafkaHandleBase::OffsetTuple
KafkaHandleBase::query_offsets(const TopicPartition& topic_partition,
milliseconds timeout) const {
int64_t low;
int64_t high;
const string& topic = topic_partition.get_topic();
const int partition = topic_partition.get_partition();
const int timeout_ms = static_cast<int>(timeout.count());
rd_kafka_resp_err_t result = rd_kafka_query_watermark_offsets(handle_.get(), topic.data(),
partition, &low, &high,
timeout_ms);
check_error(result);
return make_tuple(low, high);
}
Metadata KafkaHandleBase::get_metadata(bool all_topics) const {
return get_metadata(all_topics, nullptr, timeout_ms_);
}
Metadata KafkaHandleBase::get_metadata(bool all_topics,
milliseconds timeout) const {
return get_metadata(all_topics, nullptr, timeout);
}
TopicMetadata KafkaHandleBase::get_metadata(const Topic& topic) const {
return get_metadata(topic, timeout_ms_);
}
TopicMetadata KafkaHandleBase::get_metadata(const Topic& topic,
milliseconds timeout) const {
Metadata md = get_metadata(false, topic.get_handle(), timeout);
auto topics = md.get_topics();
if (topics.empty()) {
throw ElementNotFound("topic metadata", topic.get_name());
}
return topics.front();
}
GroupInformation KafkaHandleBase::get_consumer_group(const string& name) {
return get_consumer_group(name, timeout_ms_);
}
GroupInformation KafkaHandleBase::get_consumer_group(const string& name,
milliseconds timeout) {
auto result = fetch_consumer_groups(name.c_str(), timeout);
if (result.empty()) {
throw ElementNotFound("consumer group information", name);
}
return move(result[0]);
}
vector<GroupInformation> KafkaHandleBase::get_consumer_groups() {
return get_consumer_groups(timeout_ms_);
}
vector<GroupInformation> KafkaHandleBase::get_consumer_groups(milliseconds timeout) {
return fetch_consumer_groups(nullptr, timeout);
}
TopicPartitionList
KafkaHandleBase::get_offsets_for_times(const TopicPartitionsTimestampsMap& queries) const {
return get_offsets_for_times(queries, timeout_ms_);
}
TopicPartitionList
KafkaHandleBase::get_offsets_for_times(const TopicPartitionsTimestampsMap& queries,
milliseconds timeout) const {
TopicPartitionList topic_partitions;
for (const auto& query : queries) {
const TopicPartition& topic_partition = query.first;
topic_partitions.emplace_back(topic_partition.get_topic(), topic_partition.get_partition(),
query.second.count());
}
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
const int timeout_ms = static_cast<int>(timeout.count());
rd_kafka_resp_err_t result = rd_kafka_offsets_for_times(handle_.get(), topic_list_handle.get(),
timeout_ms);
check_error(result, topic_list_handle.get());
return convert(topic_list_handle);
}
string KafkaHandleBase::get_name() const {
return rd_kafka_name(handle_.get());
}
milliseconds KafkaHandleBase::get_timeout() const {
return timeout_ms_;
}
const Configuration& KafkaHandleBase::get_configuration() const {
return config_;
}
int KafkaHandleBase::get_out_queue_length() const {
return rd_kafka_outq_len(handle_.get());
}
void KafkaHandleBase::yield() const {
rd_kafka_yield(handle_.get());
}
void KafkaHandleBase::set_handle(rd_kafka_t* handle) {
handle_ = HandlePtr(handle, HandleDeleter(this));
}
Topic KafkaHandleBase::get_topic(const string& name, rd_kafka_topic_conf_t* conf) {
rd_kafka_topic_t* topic = rd_kafka_topic_new(get_handle(), name.data(), conf);
if (!topic) {
throw HandleException(rd_kafka_last_error());
}
return Topic(topic);
}
Metadata KafkaHandleBase::get_metadata(bool all_topics,
rd_kafka_topic_t* topic_ptr,
milliseconds timeout) const {
const rd_kafka_metadata_t* metadata;
const int timeout_ms = static_cast<int>(timeout.count());
rd_kafka_resp_err_t error = rd_kafka_metadata(get_handle(), !!all_topics,
topic_ptr, &metadata, timeout_ms);
check_error(error);
return Metadata(metadata);
}
vector<GroupInformation> KafkaHandleBase::fetch_consumer_groups(const char* name,
milliseconds timeout) {
const rd_kafka_group_list* list = nullptr;
const int timeout_ms = static_cast<int>(timeout.count());
auto result = rd_kafka_list_groups(get_handle(), name, &list, timeout_ms);
check_error(result);
// Wrap this in a unique_ptr so it gets auto deleted
using GroupHandle = std::unique_ptr<const rd_kafka_group_list,
decltype(&rd_kafka_group_list_destroy)>;
GroupHandle group_handle(list, &rd_kafka_group_list_destroy);
vector<GroupInformation> groups;
for (int i = 0; i < list->group_cnt; ++i) {
groups.emplace_back(list->groups[i]);
}
return groups;
}
void KafkaHandleBase::save_topic_config(const string& topic_name, TopicConfiguration config) {
lock_guard<mutex> _(topic_configurations_mutex_);
auto iter = topic_configurations_.emplace(topic_name, move(config)).first;
iter->second.set_as_opaque();
}
void KafkaHandleBase::check_error(rd_kafka_resp_err_t error) const {
if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
throw HandleException(error);
}
}
void KafkaHandleBase::check_error(rd_kafka_resp_err_t error,
const rd_kafka_topic_partition_list_t* list_ptr) const {
if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
throw HandleException(error);
}
if (list_ptr) {
//check if any partition has errors
for (int i = 0; i < list_ptr->cnt; ++i) {
if (list_ptr->elems[i].err != RD_KAFKA_RESP_ERR_NO_ERROR) {
throw HandleException(list_ptr->elems[i].err);
}
}
}
}
void KafkaHandleBase::check_error(rd_kafka_error_t* error,
const rd_kafka_topic_partition_list_t* list_ptr) const {
if (!error) {
return;
}
rd_kafka_resp_err_t error_code = error->code;
rd_kafka_error_destroy(error);
check_error(error_code, list_ptr);
}
rd_kafka_conf_t* KafkaHandleBase::get_configuration_handle() {
return config_.get_handle();
}
#if RD_KAFKA_VERSION >= RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION
void KafkaHandleBase::set_destroy_flags(int destroy_flags) {
destroy_flags_ = destroy_flags;
};
int KafkaHandleBase::get_destroy_flags() const {
return destroy_flags_;
};
#endif
void KafkaHandleBase::HandleDeleter::operator()(rd_kafka_t* handle) {
#if RD_KAFKA_VERSION >= RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION
rd_kafka_destroy_flags(handle, handle_base_ptr_->get_destroy_flags());
#else
rd_kafka_destroy(handle);
#endif
}
} // cppkafka