kafka.cpp 38 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076
  1. /*##############################################################################
  2. HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems®.
  3. Licensed under the Apache License, Version 2.0 (the "License");
  4. you may not use this file except in compliance with the License.
  5. You may obtain a copy of the License at
  6. http://www.apache.org/licenses/LICENSE-2.0
  7. Unless required by applicable law or agreed to in writing, software
  8. distributed under the License is distributed on an "AS IS" BASIS,
  9. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. See the License for the specific language governing permissions and
  11. limitations under the License.
  12. ############################################################################## */
  13. #include "kafka.hpp"
  14. #include "rtlds_imp.hpp"
  15. #include "jlog.hpp"
  16. #include "jmutex.hpp"
  17. #include "jprop.hpp"
  18. #include "jfile.hpp"
  19. #include "build-config.h"
  20. #include "librdkafka/rdkafka.h"
  21. #include <map>
  22. #include <fstream>
  23. #include <mutex>
  24. //==============================================================================
  25. // Kafka Interface Code
  26. //==============================================================================
  27. namespace KafkaPlugin
  28. {
  29. //--------------------------------------------------------------------------
  30. // File Constants
  31. //--------------------------------------------------------------------------
  32. // The minimum number of seconds that a cached object can live
  33. // without activity
  34. const time_t OBJECT_EXPIRE_TIMEOUT_SECONDS = 60 * 2;
  35. // The number of milliseconds given to librdkafka to perform explicit
  36. // background activity
  37. const __int32 POLL_TIMEOUT = 1000;
  38. //--------------------------------------------------------------------------
  39. // Static Variables
  40. //--------------------------------------------------------------------------
  41. static std::once_flag pubCacheInitFlag;
  42. //--------------------------------------------------------------------------
  43. // Static Methods (internal)
  44. //--------------------------------------------------------------------------
  45. /**
  46. * Look for an optional configuration file and apply any found configuration
  47. * parameters to a librdkafka configuration object.
  48. *
  49. * @param configFilePath The path to a configuration file; it is not
  50. * necessary for the file to exist
  51. * @param globalConfigPtr A pointer to the configuration object that
  52. * will receive any found parameters
  53. * @param traceLevel The current log trace level
  54. */
  55. static void applyConfig(const char* configFilePath, RdKafka::Conf* globalConfigPtr, int traceLevel)
  56. {
  57. if (configFilePath && *configFilePath && globalConfigPtr)
  58. {
  59. std::string errStr;
  60. StringBuffer fullConfigPath;
  61. fullConfigPath.append(CONFIG_DIR).append(PATHSEPSTR).append(configFilePath);
  62. Owned<IProperties> properties = createProperties(fullConfigPath.str(), true);
  63. Owned<IPropertyIterator> props = properties->getIterator();
  64. ForEach(*props)
  65. {
  66. StringBuffer key = props->getPropKey();
  67. key.trim();
  68. if (key.length() > 0 && key.charAt(0) != '#')
  69. {
  70. if (strcmp(key.str(), "metadata.broker.list") != 0)
  71. {
  72. const char* value = properties->queryProp(key);
  73. if (value && *value)
  74. {
  75. if (globalConfigPtr->set(key.str(), value, errStr) != RdKafka::Conf::CONF_OK)
  76. {
  77. DBGLOG("Kafka: Failed to set config param from file %s: '%s' = '%s'; error: '%s'", configFilePath, key.str(), value, errStr.c_str());
  78. }
  79. else if (traceLevel > 4)
  80. {
  81. DBGLOG("Kafka: Set config param from file %s: '%s' = '%s'", configFilePath, key.str(), value);
  82. }
  83. }
  84. }
  85. else
  86. {
  87. DBGLOG("Kafka: Setting '%s' ignored in config file %s", key.str(), configFilePath);
  88. }
  89. }
  90. }
  91. }
  92. }
  93. //--------------------------------------------------------------------------
  94. // Plugin Classes
  95. //--------------------------------------------------------------------------
  96. KafkaStreamedDataset::KafkaStreamedDataset(Consumer* _consumerPtr, IEngineRowAllocator* _resultAllocator, int _traceLevel, __int64 _maxRecords)
  97. : consumerPtr(_consumerPtr),
  98. resultAllocator(_resultAllocator),
  99. traceLevel(_traceLevel),
  100. maxRecords(_maxRecords)
  101. {
  102. shouldRead = true;
  103. consumedRecCount = 0;
  104. lastMsgOffset = 0;
  105. }
  106. KafkaStreamedDataset::~KafkaStreamedDataset()
  107. {
  108. if (consumedRecCount > 0)
  109. {
  110. consumerPtr->commitOffset(lastMsgOffset);
  111. }
  112. delete(consumerPtr);
  113. }
  114. const void* KafkaStreamedDataset::nextRow()
  115. {
  116. const void* result = NULL;
  117. __int32 maxAttempts = 10; //!< Maximum number of tries if local queue is full
  118. __int32 timeoutWait = 100; //!< Amount of time to wait between retries
  119. __int32 attemptNum = 0;
  120. if (maxRecords <= 0 || consumedRecCount < maxRecords)
  121. {
  122. RdKafka::Message* messageObjPtr = NULL;
  123. bool messageConsumed = false;
  124. while (!messageConsumed && shouldRead && attemptNum < maxAttempts)
  125. {
  126. messageObjPtr = consumerPtr->getOneMessage(); // messageObjPtr must be deleted when we are through with it
  127. if (messageObjPtr)
  128. {
  129. try
  130. {
  131. switch (messageObjPtr->err())
  132. {
  133. case RdKafka::ERR_NO_ERROR:
  134. {
  135. RtlDynamicRowBuilder rowBuilder(resultAllocator);
  136. unsigned len = sizeof(__int32) + sizeof(__int64) + sizeof(size32_t) + messageObjPtr->len();
  137. byte* row = rowBuilder.ensureCapacity(len, NULL);
  138. // Populating this structure:
  139. // EXPORT KafkaMessage := RECORD
  140. // UNSIGNED4 partitionNum;
  141. // UNSIGNED8 offset;
  142. // STRING message;
  143. // END;
  144. *(__int32*)(row) = messageObjPtr->partition();
  145. *(__int64*)(row + sizeof(__int32)) = messageObjPtr->offset();
  146. *(size32_t*)(row + sizeof(__int32) + sizeof(__int64)) = messageObjPtr->len();
  147. memcpy(row + sizeof(__int32) + sizeof(__int64) + sizeof(size32_t), messageObjPtr->payload(), messageObjPtr->len());
  148. result = rowBuilder.finalizeRowClear(len);
  149. lastMsgOffset = messageObjPtr->offset();
  150. ++consumedRecCount;
  151. // Give opportunity for consumer to pull in any additional messages
  152. consumerPtr->handle()->poll(0);
  153. // Mark as loaded so we don't retry
  154. messageConsumed = true;
  155. }
  156. break;
  157. case RdKafka::ERR__TIMED_OUT:
  158. // No new messages arrived and we timed out waiting
  159. ++attemptNum;
  160. consumerPtr->handle()->poll(timeoutWait);
  161. break;
  162. case RdKafka::ERR__PARTITION_EOF:
  163. // We reached the end of the messages in the partition
  164. if (traceLevel > 4)
  165. {
  166. DBGLOG("Kafka: EOF reading message from partition %d", messageObjPtr->partition());
  167. }
  168. shouldRead = false;
  169. break;
  170. case RdKafka::ERR__UNKNOWN_PARTITION:
  171. // Unknown partition; don't throw an error here because
  172. // in some configurations (e.g. more Thor slaves than
  173. // partitions) not all consumers will have a partition
  174. // to read
  175. if (traceLevel > 4)
  176. {
  177. DBGLOG("Kafka: Unknown partition while trying to read");
  178. }
  179. shouldRead = false;
  180. break;
  181. case RdKafka::ERR__UNKNOWN_TOPIC:
  182. throw MakeStringException(-1, "Kafka: Error while reading message: '%s'", messageObjPtr->errstr().c_str());
  183. break;
  184. }
  185. }
  186. catch (...)
  187. {
  188. delete(messageObjPtr);
  189. throw;
  190. }
  191. delete(messageObjPtr);
  192. messageObjPtr = NULL;
  193. }
  194. }
  195. }
  196. return result;
  197. }
  198. void KafkaStreamedDataset::stop()
  199. {
  200. shouldRead = false;
  201. }
  202. //--------------------------------------------------------------------------
  203. Poller::Poller(KafkaObj* _parentPtr, __int32 _pollTimeout)
  204. : Thread("Kafka::Poller"),
  205. parentPtr(_parentPtr),
  206. pollTimeout(_pollTimeout),
  207. shouldRun(false)
  208. {
  209. }
  210. void Poller::start()
  211. {
  212. if (!isAlive() && parentPtr)
  213. {
  214. shouldRun = true;
  215. Thread::start();
  216. }
  217. }
  218. void Poller::stop()
  219. {
  220. if (isAlive())
  221. {
  222. shouldRun = false;
  223. join();
  224. }
  225. }
  226. int Poller::run()
  227. {
  228. RdKafka::Handle* handle = parentPtr->handle();
  229. while (shouldRun)
  230. {
  231. handle->poll(pollTimeout);
  232. }
  233. return 0;
  234. }
  235. //--------------------------------------------------------------------------
  236. Publisher::Publisher(const std::string& _brokers, const std::string& _topic, __int32 _pollTimeout, int _traceLevel)
  237. : brokers(_brokers),
  238. topic(_topic),
  239. pollTimeout(_pollTimeout),
  240. traceLevel(_traceLevel)
  241. {
  242. producerPtr = NULL;
  243. topicPtr = NULL;
  244. pollerPtr = new Poller(this, _pollTimeout);
  245. updateTimeTouched();
  246. }
  247. Publisher::~Publisher()
  248. {
  249. delete(pollerPtr);
  250. delete(topicPtr.load());
  251. delete(producerPtr);
  252. }
  253. RdKafka::Handle* Publisher::handle()
  254. {
  255. return static_cast<RdKafka::Handle*>(producerPtr);
  256. }
  257. time_t Publisher::updateTimeTouched()
  258. {
  259. timeCreated = time(NULL);
  260. return timeCreated;
  261. }
  262. time_t Publisher::getTimeTouched() const
  263. {
  264. return timeCreated;
  265. }
  266. void Publisher::shutdownPoller()
  267. {
  268. if (pollerPtr)
  269. {
  270. // Wait until we send all messages
  271. while (messagesWaitingInQueue() > 0)
  272. {
  273. usleep(pollTimeout);
  274. }
  275. // Tell poller to stop
  276. pollerPtr->stop();
  277. }
  278. }
  279. __int32 Publisher::messagesWaitingInQueue()
  280. {
  281. __int32 queueLength = 0;
  282. if (producerPtr)
  283. {
  284. queueLength = producerPtr->outq_len();
  285. }
  286. return queueLength;
  287. }
  288. void Publisher::ensureSetup()
  289. {
  290. if (!topicPtr.load(std::memory_order_acquire))
  291. {
  292. CriticalBlock block(lock);
  293. if (!topicPtr.load(std::memory_order_relaxed))
  294. {
  295. std::string errStr;
  296. RdKafka::Conf* globalConfig = RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL);
  297. if (globalConfig)
  298. {
  299. // Set global configuration parameters, used mainly at the producer level
  300. globalConfig->set("metadata.broker.list", brokers, errStr);
  301. globalConfig->set("queue.buffering.max.messages", "1000000", errStr);
  302. globalConfig->set("compression.codec", "snappy", errStr);
  303. globalConfig->set("message.send.max.retries", "3", errStr);
  304. globalConfig->set("retry.backoff.ms", "500", errStr);
  305. // Set any global configurations from file, allowing
  306. // overrides of above settings
  307. applyConfig("kafka_global.conf", globalConfig, traceLevel);
  308. // Set producer callbacks
  309. globalConfig->set("event_cb", static_cast<RdKafka::EventCb*>(this), errStr);
  310. globalConfig->set("dr_cb", static_cast<RdKafka::DeliveryReportCb*>(this), errStr);
  311. // Create the producer
  312. producerPtr = RdKafka::Producer::create(globalConfig, errStr);
  313. if (producerPtr)
  314. {
  315. RdKafka::Conf* topicConfPtr = RdKafka::Conf::create(RdKafka::Conf::CONF_TOPIC);
  316. // Set any topic configurations from file
  317. std::string confName = "kafka_publisher_topic_" + topic + ".conf";
  318. applyConfig(confName.c_str(), topicConfPtr, traceLevel);
  319. // Create the topic
  320. topicPtr.store(RdKafka::Topic::create(producerPtr, topic, topicConfPtr, errStr), std::memory_order_release);
  321. if (topicPtr)
  322. {
  323. // Start the attached background poller
  324. pollerPtr->start();
  325. }
  326. else
  327. {
  328. throw MakeStringException(-1, "Kafka: Unable to create producer topic object for topic '%s'; error: '%s'", topic.c_str(), errStr.c_str());
  329. }
  330. }
  331. else
  332. {
  333. throw MakeStringException(-1, "Kafka: Unable to create producer object for brokers '%s'; error: '%s'", brokers.c_str(), errStr.c_str());
  334. }
  335. }
  336. else
  337. {
  338. throw MakeStringException(-1, "Kafka: Unable to create producer global configuration object for brokers '%s'; error: '%s'", brokers.c_str(), errStr.c_str());
  339. }
  340. }
  341. }
  342. }
  343. void Publisher::sendMessage(const std::string& message, const std::string& key)
  344. {
  345. __int32 maxAttempts = 10; //!< Maximum number of tries if local queue is full
  346. __int32 attemptNum = 0;
  347. // Make sure we have a valid connection to the Kafka cluster
  348. ensureSetup();
  349. // Actually send the message
  350. while (true)
  351. {
  352. RdKafka::ErrorCode resp = producerPtr->produce(topicPtr, RdKafka::Topic::PARTITION_UA, RdKafka::Producer::RK_MSG_COPY, const_cast<char*>(message.c_str()), message.size(), (key.empty() ? NULL : &key), NULL);
  353. if (resp == RdKafka::ERR_NO_ERROR)
  354. {
  355. break;
  356. }
  357. else if (resp == RdKafka::ERR__QUEUE_FULL)
  358. {
  359. if (attemptNum < maxAttempts)
  360. {
  361. usleep(pollTimeout);
  362. ++attemptNum;
  363. }
  364. else
  365. {
  366. throw MakeStringException(-1, "Kafka: Unable to send message to topic '%s'; error: '%s'", topic.c_str(), RdKafka::err2str(resp).c_str());
  367. }
  368. }
  369. else
  370. {
  371. throw MakeStringException(-1, "Kafka: Unable to send message to topic '%s'; error: '%s'", topic.c_str(), RdKafka::err2str(resp).c_str());
  372. }
  373. }
  374. }
  375. void Publisher::event_cb(RdKafka::Event& event)
  376. {
  377. if (traceLevel > 4)
  378. {
  379. switch (event.type())
  380. {
  381. case RdKafka::Event::EVENT_ERROR:
  382. DBGLOG("Kafka: Error: %s", event.str().c_str());
  383. break;
  384. case RdKafka::Event::EVENT_STATS:
  385. DBGLOG("Kafka: Stats: %s", event.str().c_str());
  386. break;
  387. case RdKafka::Event::EVENT_LOG:
  388. DBGLOG("Kafka: Log: %s", event.str().c_str());
  389. break;
  390. }
  391. }
  392. }
  393. void Publisher::dr_cb (RdKafka::Message& message)
  394. {
  395. if (message.err() != RdKafka::ERR_NO_ERROR)
  396. {
  397. StringBuffer payloadStr;
  398. if (message.len() == 0)
  399. payloadStr.append("<no message>");
  400. else
  401. payloadStr.append(message.len(), static_cast<const char*>(message.payload()));
  402. DBGLOG("Kafka: Error publishing message: %d (%s); message: '%s'", message.err(), message.errstr().c_str(), payloadStr.str());
  403. }
  404. }
  405. //--------------------------------------------------------------------------
  406. Consumer::Consumer(const std::string& _brokers, const std::string& _topic, const std::string& _consumerGroup, __int32 _partitionNum, int _traceLevel)
  407. : brokers(_brokers),
  408. topic(_topic),
  409. consumerGroup(_consumerGroup),
  410. partitionNum(_partitionNum),
  411. traceLevel(_traceLevel)
  412. {
  413. consumerPtr = NULL;
  414. topicPtr = NULL;
  415. }
  416. Consumer::~Consumer()
  417. {
  418. if (consumerPtr && topicPtr)
  419. {
  420. consumerPtr->stop(topicPtr, partitionNum);
  421. }
  422. delete(topicPtr.load());
  423. delete(consumerPtr);
  424. }
  425. RdKafka::Handle* Consumer::handle()
  426. {
  427. return static_cast<RdKafka::Handle*>(consumerPtr);
  428. }
  429. void Consumer::ensureSetup()
  430. {
  431. if (!topicPtr.load(std::memory_order_acquire))
  432. {
  433. CriticalBlock block(lock);
  434. if (!topicPtr.load(std::memory_order_relaxed))
  435. {
  436. initFileOffsetIfNotExist();
  437. std::string errStr;
  438. RdKafka::Conf* globalConfig = RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL);
  439. if (globalConfig)
  440. {
  441. // Set global configuration parameters, used mainly at the consumer level
  442. globalConfig->set("metadata.broker.list", brokers, errStr);
  443. globalConfig->set("compression.codec", "snappy", errStr);
  444. globalConfig->set("queued.max.messages.kbytes", "10000000", errStr);
  445. globalConfig->set("fetch.message.max.bytes", "10000000", errStr);
  446. // Set any global configurations from file, allowing
  447. // overrides of above settings
  448. applyConfig("kafka_global.conf", globalConfig, traceLevel);
  449. // Set consumer callbacks
  450. globalConfig->set("event_cb", static_cast<RdKafka::EventCb*>(this), errStr);
  451. // Create the consumer
  452. consumerPtr = RdKafka::Consumer::create(globalConfig, errStr);
  453. if (consumerPtr)
  454. {
  455. RdKafka::Conf* topicConfPtr = RdKafka::Conf::create(RdKafka::Conf::CONF_TOPIC);
  456. // Set the per-topic configuration parameters
  457. topicConfPtr->set("group.id", consumerGroup, errStr);
  458. topicConfPtr->set("auto.offset.reset", "smallest", errStr);
  459. // Set any topic configurations from file, allowing
  460. // overrides of above settings
  461. std::string confName = "kafka_consumer_topic_" + topic + ".conf";
  462. applyConfig(confName.c_str(), topicConfPtr, traceLevel);
  463. // Ensure that some items are set a certain way
  464. // by setting them after loading the external conf
  465. topicConfPtr->set("auto.commit.enable", "false", errStr);
  466. // Create the topic
  467. topicPtr.store(RdKafka::Topic::create(consumerPtr, topic, topicConfPtr, errStr), std::memory_order_release);
  468. if (!topicPtr)
  469. {
  470. throw MakeStringException(-1, "Kafka: Unable to create consumer topic object for topic '%s'; error: '%s'", topic.c_str(), errStr.c_str());
  471. }
  472. }
  473. else
  474. {
  475. throw MakeStringException(-1, "Kafka: Unable to create consumer object for brokers '%s'; error: '%s'", brokers.c_str(), errStr.c_str());
  476. }
  477. }
  478. else
  479. {
  480. throw MakeStringException(-1, "Kafka: Unable to create consumer global configuration object for brokers '%s'; error: '%s'", brokers.c_str(), errStr.c_str());
  481. }
  482. }
  483. }
  484. }
  485. RdKafka::Message* Consumer::getOneMessage()
  486. {
  487. return consumerPtr->consume(topicPtr, partitionNum, POLL_TIMEOUT);
  488. }
  489. KafkaStreamedDataset* Consumer::getMessageDataset(IEngineRowAllocator* allocator, __int64 maxRecords)
  490. {
  491. // Make sure we have a valid connection to the Kafka cluster
  492. ensureSetup();
  493. // Start the local read queue
  494. RdKafka::ErrorCode startErr = consumerPtr->start(topicPtr, partitionNum, RdKafka::Topic::OFFSET_STORED);
  495. if (startErr == RdKafka::ERR_NO_ERROR)
  496. {
  497. if (traceLevel > 4)
  498. {
  499. DBGLOG("Kafka: Started Consumer for %s:%d @ %s", topic.c_str(), partitionNum, brokers.c_str());
  500. }
  501. }
  502. else
  503. {
  504. throw MakeStringException(-1, "Kafka: Failed to start Consumer read for %s:%d @ %s; error: %d", topic.c_str(), partitionNum, brokers.c_str(), startErr);
  505. }
  506. return new KafkaStreamedDataset(this, allocator, traceLevel, maxRecords);
  507. }
  508. StringBuffer Consumer::offsetFilePath() const
  509. {
  510. StringBuffer offsetPath;
  511. offsetPath.append(topic.c_str());
  512. offsetPath.append("-");
  513. offsetPath.append(partitionNum);
  514. if (!consumerGroup.empty())
  515. {
  516. offsetPath.append("-");
  517. offsetPath.append(consumerGroup.c_str());
  518. }
  519. offsetPath.append(".offset");
  520. return offsetPath;
  521. }
  522. void Consumer::commitOffset(__int64 offset) const
  523. {
  524. if (offset >= -1)
  525. {
  526. // Not using librdkafka's offset_store because it seems to be broken
  527. // topicPtr->offset_store(partitionNum, offset);
  528. // Create/overwrite a file using the same naming convention and
  529. // file contents that librdkafka uses so it can pick up where
  530. // we left off; NOTE: librdkafka does not clean the topic name
  531. // or consumer group name when constructing this path
  532. // (which is actually a security concern), so we can't clean, either
  533. StringBuffer offsetPath = offsetFilePath();
  534. std::ofstream outFile(offsetPath.str(), std::ofstream::trunc);
  535. outFile << offset;
  536. if (traceLevel > 4)
  537. {
  538. DBGLOG("Kafka: Saved offset %lld to %s", offset, offsetPath.str());
  539. }
  540. }
  541. }
  542. void Consumer::initFileOffsetIfNotExist() const
  543. {
  544. StringBuffer offsetPath = offsetFilePath();
  545. if (!checkFileExists(offsetPath.str()))
  546. {
  547. commitOffset(-1);
  548. if (traceLevel > 4)
  549. {
  550. DBGLOG("Kafka: Creating initial offset file %s", offsetPath.str());
  551. }
  552. }
  553. }
  554. void Consumer::event_cb(RdKafka::Event& event)
  555. {
  556. if (traceLevel > 4)
  557. {
  558. switch (event.type())
  559. {
  560. case RdKafka::Event::EVENT_ERROR:
  561. DBGLOG("Kafka: Error: %s", event.str().c_str());
  562. break;
  563. case RdKafka::Event::EVENT_STATS:
  564. DBGLOG("Kafka: Stats: %s", event.str().c_str());
  565. break;
  566. case RdKafka::Event::EVENT_LOG:
  567. DBGLOG("Kafka: Log: %s", event.str().c_str());
  568. break;
  569. }
  570. }
  571. }
  572. //--------------------------------------------------------------------------
  573. /** @class PublisherCacheObj
  574. *
  575. * Class used to create and cache publisher objects and connections
  576. */
  577. static class PublisherCacheObj
  578. {
  579. private:
  580. typedef std::map<std::string, Publisher*> ObjMap;
  581. public:
  582. /**
  583. * Constructor
  584. *
  585. * @param _traceLevel The current logging level
  586. */
  587. PublisherCacheObj(int _traceLevel)
  588. : traceLevel(_traceLevel)
  589. {
  590. }
  591. void deleteAll()
  592. {
  593. CriticalBlock block(lock);
  594. for (ObjMap::iterator x = cachedPublishers.begin(); x != cachedPublishers.end(); x++)
  595. {
  596. if (x->second)
  597. {
  598. // Shutdown the attached poller before deleting
  599. x->second->shutdownPoller();
  600. // Now delete
  601. delete(x->second);
  602. }
  603. }
  604. cachedPublishers.clear();
  605. }
  606. /**
  607. * Remove previously-created objects that have been inactive
  608. * for awhile
  609. */
  610. void expire()
  611. {
  612. if (!cachedPublishers.empty())
  613. {
  614. CriticalBlock block(lock);
  615. time_t oldestAllowedTime = time(NULL) - OBJECT_EXPIRE_TIMEOUT_SECONDS;
  616. __int32 expireCount = 0;
  617. for (ObjMap::iterator x = cachedPublishers.begin(); x != cachedPublishers.end(); /* increment handled explicitly */)
  618. {
  619. // Expire only if the publisher has been inactive and if
  620. // there are no messages in the outbound queue
  621. if (x->second && x->second->getTimeTouched() < oldestAllowedTime && x->second->messagesWaitingInQueue() == 0)
  622. {
  623. // Shutdown the attached poller before deleting
  624. x->second->shutdownPoller();
  625. // Delete the object
  626. delete(x->second);
  627. // Erase from map
  628. cachedPublishers.erase(x++);
  629. ++expireCount;
  630. }
  631. else
  632. {
  633. x++;
  634. }
  635. }
  636. if (traceLevel > 4 && expireCount > 0)
  637. {
  638. DBGLOG("Kafka: Expired %d cached publisher%s", expireCount, (expireCount == 1 ? "" : "s"));
  639. }
  640. }
  641. }
  642. /**
  643. * Gets an established Publisher, based on unique broker/topic
  644. * pairs, or creates a new one.
  645. *
  646. * @param brokers One or more Kafka brokers, in the
  647. * format 'name[:port]' where 'name'
  648. * is either a host name or IP address;
  649. * multiple brokers can be delimited
  650. * with commas
  651. * @param topic The name of the topic
  652. * @param pollTimeout The number of milliseconds to give
  653. * to librdkafka when executing
  654. * asynchronous activities
  655. *
  656. * @return A pointer to a Publisher* object.
  657. */
  658. Publisher* getPublisher(const std::string& brokers, const std::string& topic, __int32 pollTimeout)
  659. {
  660. Publisher* pubObjPtr = NULL;
  661. StringBuffer suffixStr;
  662. std::string key;
  663. // Create the key used to look up previously-created objects
  664. suffixStr.append(pollTimeout);
  665. key = brokers + "+" + topic + "+" + suffixStr.str();
  666. {
  667. CriticalBlock block(lock);
  668. // Try to find a cached publisher
  669. pubObjPtr = cachedPublishers[key];
  670. if (pubObjPtr)
  671. {
  672. pubObjPtr->updateTimeTouched();
  673. }
  674. else
  675. {
  676. // Publisher for that set of brokers and topic does not exist; create one
  677. pubObjPtr = new Publisher(brokers, topic, pollTimeout, traceLevel);
  678. cachedPublishers[key] = pubObjPtr;
  679. if (traceLevel > 4)
  680. {
  681. DBGLOG("Kafka: Created and cached new publisher object: %s @ %s", topic.c_str(), brokers.c_str());
  682. }
  683. }
  684. }
  685. if (!pubObjPtr)
  686. {
  687. throw MakeStringException(-1, "Kafka: Unable to create publisher for brokers '%s' and topic '%s'", brokers.c_str(), topic.c_str());
  688. }
  689. return pubObjPtr;
  690. }
  691. private:
  692. ObjMap cachedPublishers; //!< std::map of created Publisher object pointers
  693. CriticalSection lock; //!< Mutex guarding modifications to cachedPublishers
  694. int traceLevel; //!< The current logging level
  695. } *publisherCache;
  696. //--------------------------------------------------------------------------
  697. /** @class PublisherCacheExpirerObj
  698. * Class used to expire old publisher objects held within publisherCache
  699. */
  700. static class PublisherCacheExpirerObj : public Thread
  701. {
  702. public:
  703. PublisherCacheExpirerObj()
  704. : Thread("Kafka::PublisherExpirer"),
  705. shouldRun(false)
  706. {
  707. }
  708. virtual void start()
  709. {
  710. if (!isAlive())
  711. {
  712. shouldRun = true;
  713. Thread::start();
  714. }
  715. }
  716. virtual void stop()
  717. {
  718. if (isAlive())
  719. {
  720. shouldRun = false;
  721. join();
  722. }
  723. }
  724. virtual int run()
  725. {
  726. while (shouldRun)
  727. {
  728. if (publisherCache)
  729. {
  730. publisherCache->expire();
  731. }
  732. usleep(1000);
  733. }
  734. return 0;
  735. }
  736. private:
  737. std::atomic_bool shouldRun; //!< If true, we should execute our thread's main event loop
  738. } *publisherCacheExpirer;
  739. //--------------------------------------------------------------------------
  740. // Lazy Initialization
  741. //--------------------------------------------------------------------------
  742. /**
  743. * Make sure the publisher object cache is initialized as well as the
  744. * associated background thread for expiring idle publishers. This is
  745. * called only once.
  746. *
  747. * @param traceLevel Current logging level
  748. */
  749. static void setupPublisherCache(int traceLevel)
  750. {
  751. KafkaPlugin::publisherCache = new KafkaPlugin::PublisherCacheObj(traceLevel);
  752. KafkaPlugin::publisherCacheExpirer = new KafkaPlugin::PublisherCacheExpirerObj;
  753. KafkaPlugin::publisherCacheExpirer->start();
  754. }
  755. //--------------------------------------------------------------------------
  756. // Advertised Entry Point Functions
  757. //--------------------------------------------------------------------------
  758. ECL_KAFKA_API bool ECL_KAFKA_CALL publishMessage(ICodeContext* ctx, const char* brokers, const char* topic, const char* message, const char* key)
  759. {
  760. std::call_once(pubCacheInitFlag, setupPublisherCache, ctx->queryContextLogger().queryTraceLevel());
  761. Publisher* pubObjPtr = publisherCache->getPublisher(brokers, topic, POLL_TIMEOUT);
  762. pubObjPtr->sendMessage(message, key);
  763. return true;
  764. }
  765. ECL_KAFKA_API __int32 ECL_KAFKA_CALL getTopicPartitionCount(ICodeContext* ctx, const char* brokers, const char* topic)
  766. {
  767. // We have to use librdkafka's C API for this right now, as the C++ API
  768. // does not expose a topic's metadata. In addition, there is no easy
  769. // link between the exposed C++ objects and the structs used by the
  770. // C API, so we are basically creating a brand-new connection from
  771. // scratch.
  772. __int32 pCount = 0;
  773. char errstr[512];
  774. rd_kafka_conf_t* conf = rd_kafka_conf_new();
  775. rd_kafka_t* rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr));
  776. if (rk)
  777. {
  778. if (rd_kafka_brokers_add(rk, brokers) != 0)
  779. {
  780. rd_kafka_topic_conf_t* topic_conf = rd_kafka_topic_conf_new();
  781. rd_kafka_topic_t* rkt = rd_kafka_topic_new(rk, topic, topic_conf);
  782. if (rkt)
  783. {
  784. const struct rd_kafka_metadata* metadata = NULL;
  785. rd_kafka_resp_err_t err = rd_kafka_metadata(rk, 0, rkt, &metadata, 5000);
  786. if (err == RD_KAFKA_RESP_ERR_NO_ERROR)
  787. {
  788. pCount = metadata->topics[0].partition_cnt;
  789. rd_kafka_metadata_destroy(metadata);
  790. }
  791. else
  792. {
  793. if (ctx->queryContextLogger().queryTraceLevel() > 4)
  794. {
  795. DBGLOG("Kafka: Error retrieving metadata from topic: %s @ %s: '%s'", topic, brokers, rd_kafka_err2str(err));
  796. }
  797. }
  798. rd_kafka_topic_destroy(rkt);
  799. }
  800. else
  801. {
  802. if (ctx->queryContextLogger().queryTraceLevel() > 4)
  803. {
  804. DBGLOG("Kafka: Could not create topic object: %s @ %s", topic, brokers);
  805. }
  806. }
  807. }
  808. else
  809. {
  810. if (ctx->queryContextLogger().queryTraceLevel() > 4)
  811. {
  812. DBGLOG("Kafka: Could not add brokers: %s @ %s", topic, brokers);
  813. }
  814. }
  815. rd_kafka_destroy(rk);
  816. }
  817. if (pCount == 0)
  818. {
  819. DBGLOG("Kafka: Unable to retrieve partition count from topic: %s @ %s", topic, brokers);
  820. }
  821. return pCount;
  822. }
  823. ECL_KAFKA_API IRowStream* ECL_KAFKA_CALL getMessageDataset(ICodeContext* ctx, IEngineRowAllocator* allocator, const char* brokers, const char* topic, const char* consumerGroup, __int32 partitionNum, __int64 maxRecords)
  824. {
  825. Consumer* consumerObjPtr = new Consumer(brokers, topic, consumerGroup, partitionNum, ctx->queryContextLogger().queryTraceLevel());
  826. return consumerObjPtr->getMessageDataset(allocator, maxRecords);
  827. }
  828. ECL_KAFKA_API __int64 ECL_KAFKA_CALL setMessageOffset(ICodeContext* ctx, const char* brokers, const char* topic, const char* consumerGroup, __int32 partitionNum, __int64 newOffset)
  829. {
  830. Consumer consumerObj(brokers, topic, consumerGroup, partitionNum, ctx->queryContextLogger().queryTraceLevel());
  831. consumerObj.commitOffset(newOffset);
  832. return newOffset;
  833. }
  834. }
  835. //==============================================================================
  836. // Plugin Initialization and Teardown
  837. //==============================================================================
  838. #define CURRENT_KAFKA_VERSION "kafka plugin 1.0.0"
  839. static const char* kafkaCompatibleVersions[] = {
  840. CURRENT_KAFKA_VERSION,
  841. NULL };
  842. ECL_KAFKA_API bool getECLPluginDefinition(ECLPluginDefinitionBlock* pb)
  843. {
  844. if (pb->size == sizeof(ECLPluginDefinitionBlockEx))
  845. {
  846. ECLPluginDefinitionBlockEx* pbx = static_cast<ECLPluginDefinitionBlockEx*>(pb);
  847. pbx->compatibleVersions = kafkaCompatibleVersions;
  848. }
  849. else if (pb->size != sizeof(ECLPluginDefinitionBlock))
  850. {
  851. return false;
  852. }
  853. pb->magicVersion = PLUGIN_VERSION;
  854. pb->version = CURRENT_KAFKA_VERSION;
  855. pb->moduleName = "kafka";
  856. pb->ECL = NULL;
  857. pb->flags = PLUGIN_IMPLICIT_MODULE;
  858. pb->description = "ECL plugin library for the C++ API in librdkafka++\n";
  859. return true;
  860. }
  861. MODULE_INIT(INIT_PRIORITY_STANDARD)
  862. {
  863. KafkaPlugin::publisherCache = NULL;
  864. KafkaPlugin::publisherCacheExpirer = NULL;
  865. return true;
  866. }
  867. MODULE_EXIT()
  868. {
  869. // Delete the background thread expiring items from the publisher cache
  870. // before deleting the publisher cache
  871. if (KafkaPlugin::publisherCacheExpirer)
  872. {
  873. KafkaPlugin::publisherCacheExpirer->stop();
  874. delete(KafkaPlugin::publisherCacheExpirer);
  875. KafkaPlugin::publisherCacheExpirer = NULL;
  876. }
  877. if (KafkaPlugin::publisherCache)
  878. {
  879. KafkaPlugin::publisherCache->deleteAll();
  880. delete(KafkaPlugin::publisherCache);
  881. KafkaPlugin::publisherCache = NULL;
  882. }
  883. RdKafka::wait_destroyed(3000);
  884. }