@@ -116,6 +116,7 @@ void AdminClient::Init(v8::Local<v8::Object> exports) {
116116 Nan::SetPrototypeMethod (tpl, " deleteTopic" , NodeDeleteTopic);
117117 Nan::SetPrototypeMethod (tpl, " createPartitions" , NodeCreatePartitions);
118118 Nan::SetPrototypeMethod (tpl, " deleteRecords" , NodeDeleteRecords);
119+ Nan::SetPrototypeMethod (tpl, " describeTopics" , NodeDescribeTopics);
119120
120121 // Consumer group related operations
121122 Nan::SetPrototypeMethod (tpl, " listGroups" , NodeListGroups);
@@ -812,6 +813,76 @@ Baton AdminClient::DeleteRecords(rd_kafka_DeleteRecords_t **del_records,
812813 }
813814}
814815
816+ Baton AdminClient::DescribeTopics (rd_kafka_TopicCollection_t *topics,
817+ bool include_authorized_operations,
818+ int timeout_ms,
819+ rd_kafka_event_t **event_response) {
820+ if (!IsConnected ()) {
821+ return Baton (RdKafka::ERR__STATE);
822+ }
823+
824+ {
825+ scoped_shared_write_lock lock (m_connection_lock);
826+ if (!IsConnected ()) {
827+ return Baton (RdKafka::ERR__STATE);
828+ }
829+
830+ // Make admin options to establish that we are describing topics
831+ rd_kafka_AdminOptions_t *options = rd_kafka_AdminOptions_new (
832+ m_client->c_ptr (), RD_KAFKA_ADMIN_OP_DESCRIBETOPICS);
833+
834+ if (include_authorized_operations) {
835+ rd_kafka_error_t *error =
836+ rd_kafka_AdminOptions_set_include_authorized_operations (
837+ options, include_authorized_operations);
838+ if (error) {
839+ return Baton::BatonFromErrorAndDestroy (error);
840+ }
841+ }
842+
843+ char errstr[512 ];
844+ rd_kafka_resp_err_t err = rd_kafka_AdminOptions_set_request_timeout (
845+ options, timeout_ms, errstr, sizeof (errstr));
846+ if (err != RD_KAFKA_RESP_ERR_NO_ERROR) {
847+ return Baton (static_cast <RdKafka::ErrorCode>(err), errstr);
848+ }
849+
850+ // Create queue just for this operation.
851+ rd_kafka_queue_t *rkqu = rd_kafka_queue_new (m_client->c_ptr ());
852+
853+ rd_kafka_DescribeTopics (m_client->c_ptr (), topics, options, rkqu);
854+
855+ // Poll for an event by type in that queue
856+ // DON'T destroy the event. It is the out parameter, and ownership is
857+ // the caller's.
858+ *event_response =
859+ PollForEvent (rkqu, RD_KAFKA_EVENT_DESCRIBETOPICS_RESULT, timeout_ms);
860+
861+ // Destroy the queue since we are done with it.
862+ rd_kafka_queue_destroy (rkqu);
863+
864+ // Destroy the options we just made because we polled already
865+ rd_kafka_AdminOptions_destroy (options);
866+
867+ // If we got no response from that operation, this is a failure
868+ // likely due to time out
869+ if (*event_response == NULL ) {
870+ return Baton (RdKafka::ERR__TIMED_OUT);
871+ }
872+
873+ // Now we can get the error code from the event
874+ if (rd_kafka_event_error (*event_response)) {
875+ // If we had a special error code, get out of here with it
876+ const rd_kafka_resp_err_t errcode = rd_kafka_event_error (*event_response);
877+ return Baton (static_cast <RdKafka::ErrorCode>(errcode));
878+ }
879+
880+ // At this point, event_response contains the result, which needs
881+ // to be parsed/converted by the caller.
882+ return Baton (RdKafka::ERR_NO_ERROR);
883+ }
884+ }
885+
815886void AdminClient::ActivateDispatchers () {
816887 // Listen to global config
817888 m_gconfig->listen ();
@@ -1298,4 +1369,59 @@ NAN_METHOD(AdminClient::NodeDeleteRecords) {
12981369 callback, client, delete_records, 1 , operation_timeout_ms, timeout_ms));
12991370}
13001371
1372+ /* *
1373+ * Describe Topics.
1374+ */
1375+ NAN_METHOD (AdminClient::NodeDescribeTopics) {
1376+ Nan::HandleScope scope;
1377+
1378+ if (info.Length () < 3 || !info[2 ]->IsFunction ()) {
1379+ return Nan::ThrowError (" Need to specify a callback" );
1380+ }
1381+
1382+ if (!info[0 ]->IsArray ()) {
1383+ return Nan::ThrowError (" Must provide an array of 'topicNames'" );
1384+ }
1385+
1386+ v8::Local<v8::Array> topicNames = info[0 ].As <v8::Array>();
1387+
1388+ if (topicNames->Length () == 0 ) {
1389+ return Nan::ThrowError (" 'topicNames' cannot be empty" );
1390+ }
1391+
1392+ std::vector<std::string> topicNamesVector = v8ArrayToStringVector (topicNames);
1393+
1394+ const char **topics = static_cast <const char **>(
1395+ malloc (sizeof (const char *) * topicNamesVector.size ()));
1396+
1397+ for (size_t i = 0 ; i < topicNamesVector.size (); i++) {
1398+ topics[i] = topicNamesVector[i].c_str ();
1399+ }
1400+
1401+ /* *
1402+ * The ownership of this is taken by
1403+ * Workers::AdminClientDescribeTopics and freeing it is also handled
1404+ * by that class.
1405+ */
1406+ rd_kafka_TopicCollection_t *topic_collection =
1407+ rd_kafka_TopicCollection_of_topic_names (topics, topicNamesVector.size ());
1408+
1409+ free (topics);
1410+
1411+ v8::Local<v8::Object> options = info[1 ].As <v8::Object>();
1412+
1413+ bool include_authorised_operations =
1414+ GetParameter<bool >(options, " includeAuthorizedOperations" , false );
1415+
1416+ int timeout_ms = GetParameter<int64_t >(options, " timeout" , 5000 );
1417+
1418+ v8::Local<v8::Function> cb = info[2 ].As <v8::Function>();
1419+ Nan::Callback *callback = new Nan::Callback (cb);
1420+ AdminClient *client = ObjectWrap::Unwrap<AdminClient>(info.This ());
1421+
1422+ Nan::AsyncQueueWorker (new Workers::AdminClientDescribeTopics (
1423+ callback, client, topic_collection,
1424+ include_authorised_operations, timeout_ms));
1425+ }
1426+
13011427} // namespace NodeKafka
0 commit comments