-
Notifications
You must be signed in to change notification settings - Fork 402
/
Copy pathadmin.cc
624 lines (492 loc) · 18.7 KB
/
admin.cc
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
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
/*
* node-rdkafka - Node.js wrapper for RdKafka C/C++ library
*
* Copyright (c) 2016 Blizzard Entertainment
*
* This software may be modified and distributed under the terms
* of the MIT license. See the LICENSE.txt file for details.
*/
#include <string>
#include <vector>
#include <math.h>
#include "src/per-isolate-data.h"
#include "src/workers.h"
#include "src/admin.h"
using Nan::FunctionCallbackInfo;
namespace NodeKafka {
/**
* @brief AdminClient v8 wrapped object.
*
* Specializes the connection to wrap a consumer object through compositional
* inheritence. Establishes its prototype in node through `Init`
*
* @sa RdKafka::Handle
* @sa NodeKafka::Client
*/
AdminClient::AdminClient(Conf* gconfig):
Connection(gconfig, NULL) {
rkqu = NULL;
}
void AdminClient::delete_instance(void* arg) {
delete (static_cast<AdminClient*>(arg));
}
AdminClient::~AdminClient() {
Disconnect();
}
Baton AdminClient::Connect() {
std::string errstr;
{
scoped_shared_write_lock lock(m_connection_lock);
m_client = RdKafka::Producer::create(m_gconfig, errstr);
}
if (!m_client || !errstr.empty()) {
return Baton(RdKafka::ERR__STATE, errstr);
}
if (m_init_oauthToken) {
scoped_shared_write_lock lock(m_connection_lock);
if (m_init_oauthToken) {
std::list<std::string> emptyList;
std::string token = m_init_oauthToken->token;
int64_t expiry = m_init_oauthToken->expiry;
// needed for initial connection only
m_init_oauthToken.reset();
RdKafka::ErrorCode err = m_client->oauthbearer_set_token(token, expiry,
"", emptyList, errstr);
if (err != RdKafka::ERR_NO_ERROR) {
return Baton(err, errstr);
}
}
}
if (rkqu == NULL) {
rkqu = rd_kafka_queue_new(m_client->c_ptr());
}
return Baton(RdKafka::ERR_NO_ERROR);
}
Baton AdminClient::Disconnect() {
if (IsConnected()) {
scoped_shared_write_lock lock(m_connection_lock);
if (rkqu != NULL) {
rd_kafka_queue_destroy(rkqu);
rkqu = NULL;
}
delete m_client;
m_client = NULL;
}
return Baton(RdKafka::ERR_NO_ERROR);
}
void AdminClient::Init(v8::Local<v8::Object> exports) {
Nan::HandleScope scope;
v8::Local<v8::FunctionTemplate> tpl = Nan::New<v8::FunctionTemplate>(New);
tpl->SetClassName(Nan::New("AdminClient").ToLocalChecked());
tpl->InstanceTemplate()->SetInternalFieldCount(1);
// Admin client operations
Nan::SetPrototypeMethod(tpl, "createTopic", NodeCreateTopic);
Nan::SetPrototypeMethod(tpl, "deleteTopic", NodeDeleteTopic);
Nan::SetPrototypeMethod(tpl, "createPartitions", NodeCreatePartitions);
Nan::SetPrototypeMethod(tpl, "connect", NodeConnect);
Nan::SetPrototypeMethod(tpl, "disconnect", NodeDisconnect);
Nan::SetPrototypeMethod(tpl, "setToken", NodeSetToken);
PerIsolateData::For(v8::Isolate::GetCurrent())->AdminClientConstructor().Reset(
(tpl->GetFunction(Nan::GetCurrentContext())).ToLocalChecked());
Nan::Set(exports, Nan::New("AdminClient").ToLocalChecked(),
tpl->GetFunction(Nan::GetCurrentContext()).ToLocalChecked());
}
void AdminClient::New(const Nan::FunctionCallbackInfo<v8::Value>& info) {
if (!info.IsConstructCall()) {
return Nan::ThrowError("non-constructor invocation not supported");
}
if (info.Length() < 1) {
return Nan::ThrowError("You must supply a global configuration");
}
if (!info[0]->IsObject()) {
return Nan::ThrowError("Global configuration data must be specified");
}
std::string errstr;
Conf* gconfig =
Conf::create(RdKafka::Conf::CONF_GLOBAL,
(info[0]->ToObject(Nan::GetCurrentContext())).ToLocalChecked(), errstr);
if (!gconfig) {
return Nan::ThrowError(errstr.c_str());
}
AdminClient* client = new AdminClient(gconfig);
// Wrap it
client->Wrap(info.This());
// Then there is some weird initialization that happens
// basically it sets the configuration data
// we don't need to do that because we lazy load it
info.GetReturnValue().Set(info.This());
}
v8::Local<v8::Object> AdminClient::NewInstance(v8::Local<v8::Value> arg) {
Nan::EscapableHandleScope scope;
const unsigned argc = 1;
v8::Local<v8::Value> argv[argc] = { arg };
v8::Local<v8::Function> cons = Nan::New<v8::Function>(
PerIsolateData::For(v8::Isolate::GetCurrent())->AdminClientConstructor());
v8::Local<v8::Object> instance =
Nan::NewInstance(cons, argc, argv).ToLocalChecked();
return scope.Escape(instance);
}
/**
* Poll for a particular event on a queue.
*
* This will keep polling until it gets an event of that type,
* given the number of tries and a timeout
*/
rd_kafka_event_t* PollForEvent(
rd_kafka_queue_t * topic_rkqu,
rd_kafka_event_type_t event_type,
int timeout_ms) {
// Initiate exponential timeout
int attempts = 1;
int exp_timeout_ms = timeout_ms;
if (timeout_ms > 2000) {
// measure optimal number of attempts
attempts = log10(timeout_ms / 1000) / log10(2) + 1;
// measure initial exponential timeout based on attempts
exp_timeout_ms = timeout_ms / (pow(2, attempts) - 1);
}
rd_kafka_event_t * event_response = nullptr;
// Poll the event queue until we get it
do {
// free previously fetched event
rd_kafka_event_destroy(event_response);
// poll and update attempts and exponential timeout
event_response = rd_kafka_queue_poll(topic_rkqu, exp_timeout_ms);
attempts = attempts - 1;
exp_timeout_ms = 2 * exp_timeout_ms;
} while (
rd_kafka_event_type(event_response) != event_type &&
attempts > 0);
// If this isn't the type of response we want, or if we do not have a response
// type, bail out with a null
if (event_response == NULL ||
rd_kafka_event_type(event_response) != event_type) {
rd_kafka_event_destroy(event_response);
return NULL;
}
return event_response;
}
Baton AdminClient::CreateTopic(rd_kafka_NewTopic_t* topic, int timeout_ms) {
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
{
scoped_shared_write_lock lock(m_connection_lock);
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
// Make admin options to establish that we are creating topics
rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new(
m_client->c_ptr(), RD_KAFKA_ADMIN_OP_CREATETOPICS);
// Create queue just for this operation
rd_kafka_queue_t * topic_rkqu = rd_kafka_queue_new(m_client->c_ptr());
rd_kafka_CreateTopics(m_client->c_ptr(), &topic, 1, options, topic_rkqu);
// Poll for an event by type in that queue
rd_kafka_event_t * event_response = PollForEvent(
topic_rkqu,
RD_KAFKA_EVENT_CREATETOPICS_RESULT,
timeout_ms);
// Destroy the queue since we are done with it.
rd_kafka_queue_destroy(topic_rkqu);
// Destroy the options we just made because we polled already
rd_kafka_AdminOptions_destroy(options);
// If we got no response from that operation, this is a failure
// likely due to time out
if (event_response == NULL) {
return Baton(RdKafka::ERR__TIMED_OUT);
}
// Now we can get the error code from the event
if (rd_kafka_event_error(event_response)) {
// If we had a special error code, get out of here with it
const rd_kafka_resp_err_t errcode = rd_kafka_event_error(event_response);
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
// get the created results
const rd_kafka_CreateTopics_result_t * create_topic_results =
rd_kafka_event_CreateTopics_result(event_response);
size_t created_topic_count;
const rd_kafka_topic_result_t **restopics = rd_kafka_CreateTopics_result_topics( // NOLINT
create_topic_results,
&created_topic_count);
for (int i = 0 ; i < static_cast<int>(created_topic_count) ; i++) {
const rd_kafka_topic_result_t *terr = restopics[i];
const rd_kafka_resp_err_t errcode = rd_kafka_topic_result_error(terr);
const char *errmsg = rd_kafka_topic_result_error_string(terr);
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
if (errmsg) {
const std::string errormsg = std::string(errmsg);
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode), errormsg); // NOLINT
} else {
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
}
}
rd_kafka_event_destroy(event_response);
return Baton(RdKafka::ERR_NO_ERROR);
}
}
Baton AdminClient::DeleteTopic(rd_kafka_DeleteTopic_t* topic, int timeout_ms) {
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
{
scoped_shared_write_lock lock(m_connection_lock);
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
// Make admin options to establish that we are deleting topics
rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new(
m_client->c_ptr(), RD_KAFKA_ADMIN_OP_DELETETOPICS);
// Create queue just for this operation.
// May be worth making a "scoped queue" class or something like a lock
// for RAII
rd_kafka_queue_t * topic_rkqu = rd_kafka_queue_new(m_client->c_ptr());
rd_kafka_DeleteTopics(m_client->c_ptr(), &topic, 1, options, topic_rkqu);
// Poll for an event by type in that queue
rd_kafka_event_t * event_response = PollForEvent(
topic_rkqu,
RD_KAFKA_EVENT_DELETETOPICS_RESULT,
timeout_ms);
// Destroy the queue since we are done with it.
rd_kafka_queue_destroy(topic_rkqu);
// Destroy the options we just made because we polled already
rd_kafka_AdminOptions_destroy(options);
// If we got no response from that operation, this is a failure
// likely due to time out
if (event_response == NULL) {
return Baton(RdKafka::ERR__TIMED_OUT);
}
// Now we can get the error code from the event
if (rd_kafka_event_error(event_response)) {
// If we had a special error code, get out of here with it
const rd_kafka_resp_err_t errcode = rd_kafka_event_error(event_response);
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
// get the created results
const rd_kafka_DeleteTopics_result_t * delete_topic_results =
rd_kafka_event_DeleteTopics_result(event_response);
size_t deleted_topic_count;
const rd_kafka_topic_result_t **restopics = rd_kafka_DeleteTopics_result_topics( // NOLINT
delete_topic_results,
&deleted_topic_count);
for (int i = 0 ; i < static_cast<int>(deleted_topic_count) ; i++) {
const rd_kafka_topic_result_t *terr = restopics[i];
const rd_kafka_resp_err_t errcode = rd_kafka_topic_result_error(terr);
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
}
rd_kafka_event_destroy(event_response);
return Baton(RdKafka::ERR_NO_ERROR);
}
}
Baton AdminClient::CreatePartitions(
rd_kafka_NewPartitions_t* partitions,
int timeout_ms) {
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
{
scoped_shared_write_lock lock(m_connection_lock);
if (!IsConnected()) {
return Baton(RdKafka::ERR__STATE);
}
// Make admin options to establish that we are deleting topics
rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new(
m_client->c_ptr(), RD_KAFKA_ADMIN_OP_CREATEPARTITIONS);
// Create queue just for this operation.
// May be worth making a "scoped queue" class or something like a lock
// for RAII
rd_kafka_queue_t * topic_rkqu = rd_kafka_queue_new(m_client->c_ptr());
rd_kafka_CreatePartitions(m_client->c_ptr(),
&partitions, 1, options, topic_rkqu);
// Poll for an event by type in that queue
rd_kafka_event_t * event_response = PollForEvent(
topic_rkqu,
RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT,
timeout_ms);
// Destroy the queue since we are done with it.
rd_kafka_queue_destroy(topic_rkqu);
// Destroy the options we just made because we polled already
rd_kafka_AdminOptions_destroy(options);
// If we got no response from that operation, this is a failure
// likely due to time out
if (event_response == NULL) {
return Baton(RdKafka::ERR__TIMED_OUT);
}
// Now we can get the error code from the event
if (rd_kafka_event_error(event_response)) {
// If we had a special error code, get out of here with it
const rd_kafka_resp_err_t errcode = rd_kafka_event_error(event_response);
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
// get the created results
const rd_kafka_CreatePartitions_result_t * create_partitions_results =
rd_kafka_event_CreatePartitions_result(event_response);
size_t created_partitions_topic_count;
const rd_kafka_topic_result_t **restopics = rd_kafka_CreatePartitions_result_topics( // NOLINT
create_partitions_results,
&created_partitions_topic_count);
for (int i = 0 ; i < static_cast<int>(created_partitions_topic_count) ; i++) { // NOLINT
const rd_kafka_topic_result_t *terr = restopics[i];
const rd_kafka_resp_err_t errcode = rd_kafka_topic_result_error(terr);
const char *errmsg = rd_kafka_topic_result_error_string(terr);
if (errcode != RD_KAFKA_RESP_ERR_NO_ERROR) {
if (errmsg) {
const std::string errormsg = std::string(errmsg);
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode), errormsg); // NOLINT
} else {
rd_kafka_event_destroy(event_response);
return Baton(static_cast<RdKafka::ErrorCode>(errcode));
}
}
}
rd_kafka_event_destroy(event_response);
return Baton(RdKafka::ERR_NO_ERROR);
}
}
void AdminClient::ActivateDispatchers() {
// Listen to global config
m_gconfig->listen();
// Listen to non global config
// tconfig->listen();
// This should be refactored to config based management
m_event_cb.dispatcher.Activate();
}
void AdminClient::DeactivateDispatchers() {
// Stop listening to the config dispatchers
m_gconfig->stop();
// Also this one
m_event_cb.dispatcher.Deactivate();
}
/**
* @section
* C++ Exported prototype functions
*/
NAN_METHOD(AdminClient::NodeConnect) {
Nan::HandleScope scope;
AdminClient* client = ObjectWrap::Unwrap<AdminClient>(info.This());
Baton b = client->Connect();
// Let the JS library throw if we need to so the error can be more rich
int error_code = static_cast<int>(b.err());
return info.GetReturnValue().Set(Nan::New<v8::Number>(error_code));
}
NAN_METHOD(AdminClient::NodeDisconnect) {
Nan::HandleScope scope;
AdminClient* client = ObjectWrap::Unwrap<AdminClient>(info.This());
Baton b = client->Disconnect();
// Let the JS library throw if we need to so the error can be more rich
int error_code = static_cast<int>(b.err());
return info.GetReturnValue().Set(Nan::New<v8::Number>(error_code));
}
/**
* Create topic
*/
NAN_METHOD(AdminClient::NodeCreateTopic) {
Nan::HandleScope scope;
if (info.Length() < 3 || !info[2]->IsFunction()) {
// Just throw an exception
return Nan::ThrowError("Need to specify a callback");
}
if (!info[1]->IsNumber()) {
return Nan::ThrowError("Must provide 'timeout'");
}
// Create the final callback object
v8::Local<v8::Function> cb = info[2].As<v8::Function>();
Nan::Callback *callback = new Nan::Callback(cb);
AdminClient* client = ObjectWrap::Unwrap<AdminClient>(info.This());
// Get the timeout
int timeout = Nan::To<int32_t>(info[1]).FromJust();
std::string errstr;
// Get that topic we want to create
rd_kafka_NewTopic_t* topic = Conversion::Admin::FromV8TopicObject(
info[0].As<v8::Object>(), errstr);
if (topic == NULL) {
Nan::ThrowError(errstr.c_str());
return;
}
// Queue up dat work
Nan::AsyncQueueWorker(
new Workers::AdminClientCreateTopic(callback, client, topic, timeout));
return info.GetReturnValue().Set(Nan::Null());
}
/**
* Delete topic
*/
NAN_METHOD(AdminClient::NodeDeleteTopic) {
Nan::HandleScope scope;
if (info.Length() < 3 || !info[2]->IsFunction()) {
// Just throw an exception
return Nan::ThrowError("Need to specify a callback");
}
if (!info[1]->IsNumber() || !info[0]->IsString()) {
return Nan::ThrowError("Must provide 'timeout', and 'topicName'");
}
// Create the final callback object
v8::Local<v8::Function> cb = info[2].As<v8::Function>();
Nan::Callback *callback = new Nan::Callback(cb);
AdminClient* client = ObjectWrap::Unwrap<AdminClient>(info.This());
// Get the topic name from the string
std::string topic_name = Util::FromV8String(
Nan::To<v8::String>(info[0]).ToLocalChecked());
// Get the timeout
int timeout = Nan::To<int32_t>(info[1]).FromJust();
// Get that topic we want to create
rd_kafka_DeleteTopic_t* topic = rd_kafka_DeleteTopic_new(
topic_name.c_str());
// Queue up dat work
Nan::AsyncQueueWorker(
new Workers::AdminClientDeleteTopic(callback, client, topic, timeout));
return info.GetReturnValue().Set(Nan::Null());
}
/**
* Delete topic
*/
NAN_METHOD(AdminClient::NodeCreatePartitions) {
Nan::HandleScope scope;
if (info.Length() < 4) {
// Just throw an exception
return Nan::ThrowError("Need to specify a callback");
}
if (!info[3]->IsFunction()) {
// Just throw an exception
return Nan::ThrowError("Need to specify a callback 2");
}
if (!info[2]->IsNumber() || !info[1]->IsNumber() || !info[0]->IsString()) {
return Nan::ThrowError(
"Must provide 'totalPartitions', 'timeout', and 'topicName'");
}
// Create the final callback object
v8::Local<v8::Function> cb = info[3].As<v8::Function>();
Nan::Callback *callback = new Nan::Callback(cb);
AdminClient* client = ObjectWrap::Unwrap<AdminClient>(info.This());
// Get the timeout
int timeout = Nan::To<int32_t>(info[2]).FromJust();
// Get the total number of desired partitions
int partition_total_count = Nan::To<int32_t>(info[1]).FromJust();
// Get the topic name from the string
std::string topic_name = Util::FromV8String(
Nan::To<v8::String>(info[0]).ToLocalChecked());
// Create an error buffer we can throw
char* errbuf = reinterpret_cast<char*>(malloc(100));
// Create the new partitions request
rd_kafka_NewPartitions_t* new_partitions = rd_kafka_NewPartitions_new(
topic_name.c_str(), partition_total_count, errbuf, 100);
// If we got a failure on the create new partitions request,
// fail here
if (new_partitions == NULL) {
return Nan::ThrowError(errbuf);
}
// Queue up dat work
Nan::AsyncQueueWorker(new Workers::AdminClientCreatePartitions(
callback, client, new_partitions, timeout));
return info.GetReturnValue().Set(Nan::Null());
}
} // namespace NodeKafka