@@ -87,6 +87,9 @@ void AdminClient::Init(v8::Local<v8::Object> exports) {
8787 Nan::SetPrototypeMethod (tpl, " deleteTopic" , NodeDeleteTopic);
8888 Nan::SetPrototypeMethod (tpl, " createPartitions" , NodeCreatePartitions);
8989
90+ // Consumer group related operations
91+ Nan::SetPrototypeMethod (tpl, " listGroups" , NodeListGroups);
92+
9093 Nan::SetPrototypeMethod (tpl, " connect" , NodeConnect);
9194 Nan::SetPrototypeMethod (tpl, " disconnect" , NodeDisconnect);
9295 Nan::SetPrototypeMethod (tpl, " setSaslCredentials" , NodeSetSaslCredentials);
@@ -179,6 +182,9 @@ rd_kafka_event_t* PollForEvent(
179182 rd_kafka_event_type (event_response) != event_type &&
180183 attempts > 0 );
181184
185+ // TODO: change this function so a type mismatch leads to an INVALID_TYPE
186+ // error rather than a null event. A null event is treated as a timeout, which
187+ // isn't true all the time.
182188 // If this isn't the type of response we want, or if we do not have a response
183189 // type, bail out with a null
184190 if (event_response == NULL ||
@@ -421,6 +427,69 @@ Baton AdminClient::CreatePartitions(
421427 }
422428}
423429
430+ Baton AdminClient::ListGroups (
431+ bool is_match_states_set,
432+ std::vector<rd_kafka_consumer_group_state_t > &match_states, int timeout_ms,
433+ /* out */ rd_kafka_event_t **event_response) {
434+ if (!IsConnected ()) {
435+ return Baton (RdKafka::ERR__STATE);
436+ }
437+
438+ {
439+ scoped_shared_write_lock lock (m_connection_lock);
440+ if (!IsConnected ()) {
441+ return Baton (RdKafka::ERR__STATE);
442+ }
443+
444+ // Make admin options to establish that we are listing groups
445+ rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new (
446+ m_client->c_ptr (), RD_KAFKA_ADMIN_OP_LISTCONSUMERGROUPS);
447+
448+ if (is_match_states_set) {
449+ rd_kafka_error_t *error =
450+ rd_kafka_AdminOptions_set_match_consumer_group_states (
451+ options, &match_states[0 ], match_states.size ());
452+ if (error) {
453+ return Baton::BatonFromErrorAndDestroy (error);
454+ }
455+ }
456+
457+ // Create queue just for this operation.
458+ rd_kafka_queue_t *topic_rkqu = rd_kafka_queue_new (m_client->c_ptr ());
459+
460+ rd_kafka_ListConsumerGroups (m_client->c_ptr (), options, topic_rkqu);
461+
462+ // Poll for an event by type in that queue
463+ // DON'T destroy the event. It is the out parameter, and ownership is
464+ // the caller's.
465+ *event_response = PollForEvent (
466+ topic_rkqu, RD_KAFKA_EVENT_LISTCONSUMERGROUPS_RESULT, timeout_ms);
467+
468+ // Destroy the queue since we are done with it.
469+ rd_kafka_queue_destroy (topic_rkqu);
470+
471+ // Destroy the options we just made because we polled already
472+ rd_kafka_AdminOptions_destroy (options);
473+
474+ // If we got no response from that operation, this is a failure
475+ // likely due to time out
476+ if (*event_response == NULL ) {
477+ return Baton (RdKafka::ERR__TIMED_OUT);
478+ }
479+
480+ // Now we can get the error code from the event
481+ if (rd_kafka_event_error (*event_response)) {
482+ // If we had a special error code, get out of here with it
483+ const rd_kafka_resp_err_t errcode = rd_kafka_event_error (*event_response);
484+ return Baton (static_cast <RdKafka::ErrorCode>(errcode));
485+ }
486+
487+ // At this point, event_response contains the result, which needs
488+ // to be parsed/converted by the caller.
489+ return Baton (RdKafka::ERR_NO_ERROR);
490+ }
491+ }
492+
424493void AdminClient::ActivateDispatchers () {
425494 // Listen to global config
426495 m_gconfig->listen ();
@@ -600,4 +669,48 @@ NAN_METHOD(AdminClient::NodeCreatePartitions) {
600669 return info.GetReturnValue ().Set (Nan::Null ());
601670}
602671
672+ /* *
673+ * List Consumer Groups.
674+ */
675+ NAN_METHOD (AdminClient::NodeListGroups) {
676+ Nan::HandleScope scope;
677+
678+ if (info.Length () < 2 || !info[1 ]->IsFunction ()) {
679+ // Just throw an exception
680+ return Nan::ThrowError (" Need to specify a callback" );
681+ }
682+
683+ if (!info[0 ]->IsObject ()) {
684+ return Nan::ThrowError (" Must provide options object" );
685+ }
686+
687+ v8::Local<v8::Object> config = info[0 ].As <v8::Object>();
688+
689+ // Create the final callback object
690+ v8::Local<v8::Function> cb = info[1 ].As <v8::Function>();
691+ Nan::Callback *callback = new Nan::Callback (cb);
692+ AdminClient *client = ObjectWrap::Unwrap<AdminClient>(info.This ());
693+
694+ // Get the timeout - default 5000.
695+ int timeout_ms = GetParameter<int64_t >(config, " timeout" , 5000 );
696+
697+ // Get the match states, or not if they are unset.
698+ std::vector<rd_kafka_consumer_group_state_t > match_states;
699+ v8::Local<v8::String> matchConsumerGroupStatesKey =
700+ Nan::New (" matchConsumerGroupStates" ).ToLocalChecked ();
701+ bool is_match_states_set =
702+ Nan::Has (config, matchConsumerGroupStatesKey).FromMaybe (false );
703+ v8::Local<v8::Array> match_states_array;
704+
705+ if (is_match_states_set) {
706+ match_states_array = GetParameter<v8::Local<v8::Array>>(
707+ config, " matchConsumerGroupStates" , match_states_array);
708+ match_states = Conversion::Admin::FromV8GroupStateArray (match_states_array);
709+ }
710+
711+ // Queue the work.
712+ Nan::AsyncQueueWorker (new Workers::AdminClientListGroups (
713+ callback, client, is_match_states_set, match_states, timeout_ms));
714+ }
715+
603716} // namespace NodeKafka
0 commit comments