hdfs.cc 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. #include "hdfspp/hdfspp.h"
  19. #include "fs/filesystem.h"
  20. #include "common/hdfs_configuration.h"
  21. #include "common/configuration_loader.h"
  22. #include "common/logging.h"
  23. #include <hdfs/hdfs.h>
  24. #include <hdfspp/hdfs_ext.h>
  25. #include <string>
  26. #include <cstring>
  27. #include <iostream>
  28. #include <algorithm>
  29. #include <functional>
  30. using namespace hdfs;
  31. using std::experimental::nullopt;
  32. using namespace std::placeholders;
  33. static constexpr tPort kDefaultPort = 8020;
  34. /* Separate the handles used by the C api from the C++ API*/
  35. struct hdfs_internal {
  36. hdfs_internal(FileSystem *p) : filesystem_(p) {}
  37. hdfs_internal(std::unique_ptr<FileSystem> p)
  38. : filesystem_(std::move(p)) {}
  39. virtual ~hdfs_internal(){};
  40. FileSystem *get_impl() { return filesystem_.get(); }
  41. const FileSystem *get_impl() const { return filesystem_.get(); }
  42. private:
  43. std::unique_ptr<FileSystem> filesystem_;
  44. };
  45. struct hdfsFile_internal {
  46. hdfsFile_internal(FileHandle *p) : file_(p) {}
  47. hdfsFile_internal(std::unique_ptr<FileHandle> p) : file_(std::move(p)) {}
  48. virtual ~hdfsFile_internal(){};
  49. FileHandle *get_impl() { return file_.get(); }
  50. const FileHandle *get_impl() const { return file_.get(); }
  51. private:
  52. std::unique_ptr<FileHandle> file_;
  53. };
  54. /* Keep thread local copy of last error string */
  55. thread_local std::string errstr;
  56. /* Fetch last error that happened in this thread */
  57. void hdfsGetLastError(char *buf, int len) {
  58. if(nullptr == buf || len < 1) {
  59. return;
  60. }
  61. /* leave space for a trailing null */
  62. size_t copylen = std::min((size_t)errstr.size(), (size_t)len);
  63. if(copylen == (size_t)len) {
  64. copylen--;
  65. }
  66. strncpy(buf, errstr.c_str(), copylen);
  67. /* stick in null */
  68. buf[copylen] = 0;
  69. }
  70. /* Event callbacks for next open calls */
  71. thread_local std::experimental::optional<fs_event_callback> fsEventCallback;
  72. thread_local std::experimental::optional<file_event_callback> fileEventCallback;
  73. struct hdfsBuilder {
  74. hdfsBuilder();
  75. hdfsBuilder(const char * directory);
  76. virtual ~hdfsBuilder() {}
  77. ConfigurationLoader loader;
  78. HdfsConfiguration config;
  79. optional<std::string> overrideHost;
  80. optional<tPort> overridePort;
  81. optional<std::string> user;
  82. static constexpr tPort kUseDefaultPort = 0;
  83. };
  84. /* Error handling with optional debug to stderr */
  85. static void ReportError(int errnum, const std::string & msg) {
  86. errno = errnum;
  87. errstr = msg;
  88. #ifdef LIBHDFSPP_C_API_ENABLE_DEBUG
  89. std::cerr << "Error: errno=" << strerror(errnum) << " message=\"" << msg
  90. << "\"" << std::endl;
  91. #else
  92. (void)msg;
  93. #endif
  94. }
  95. /* Convert Status wrapped error into appropriate errno and return code */
  96. static int Error(const Status &stat) {
  97. const char * default_message;
  98. int errnum;
  99. int code = stat.code();
  100. switch (code) {
  101. case Status::Code::kOk:
  102. return 0;
  103. case Status::Code::kInvalidArgument:
  104. errnum = EINVAL;
  105. default_message = "Invalid argument";
  106. break;
  107. case Status::Code::kResourceUnavailable:
  108. errnum = EAGAIN;
  109. default_message = "Resource temporarily unavailable";
  110. break;
  111. case Status::Code::kUnimplemented:
  112. errnum = ENOSYS;
  113. default_message = "Function not implemented";
  114. break;
  115. case Status::Code::kException:
  116. errnum = EINTR;
  117. default_message = "Exception raised";
  118. break;
  119. case Status::Code::kOperationCanceled:
  120. errnum = EINTR;
  121. default_message = "Operation canceled";
  122. break;
  123. case Status::Code::kPermissionDenied:
  124. errnum = EACCES;
  125. default_message = "Permission denied";
  126. break;
  127. default:
  128. errnum = ENOSYS;
  129. default_message = "Error: unrecognised code";
  130. }
  131. if (stat.ToString().empty())
  132. ReportError(errnum, default_message);
  133. else
  134. ReportError(errnum, stat.ToString());
  135. return -1;
  136. }
  137. static int ReportException(const std::exception & e)
  138. {
  139. return Error(Status::Exception("Uncaught exception", e.what()));
  140. }
  141. static int ReportCaughtNonException()
  142. {
  143. return Error(Status::Exception("Uncaught value not derived from std::exception", ""));
  144. }
  145. /* return false on failure */
  146. bool CheckSystemAndHandle(hdfsFS fs, hdfsFile file) {
  147. if (!fs) {
  148. ReportError(ENODEV, "Cannot perform FS operations with null FS handle.");
  149. return false;
  150. }
  151. if (!file) {
  152. ReportError(EBADF, "Cannot perform FS operations with null File handle.");
  153. return false;
  154. }
  155. return true;
  156. }
  157. /**
  158. * C API implementations
  159. **/
  160. int hdfsFileIsOpenForRead(hdfsFile file) {
  161. /* files can only be open for reads at the moment, do a quick check */
  162. if (file) {
  163. return 1; // Update implementation when we get file writing
  164. }
  165. return 0;
  166. }
  167. hdfsFS doHdfsConnect(optional<std::string> nn, optional<tPort> port, optional<std::string> user, const Options & options) {
  168. try
  169. {
  170. IoService * io_service = IoService::New();
  171. FileSystem *fs = FileSystem::New(io_service, user.value_or(""), options);
  172. if (!fs) {
  173. ReportError(ENODEV, "Could not create FileSystem object");
  174. return nullptr;
  175. }
  176. if (fsEventCallback) {
  177. fs->SetFsEventCallback(fsEventCallback.value());
  178. }
  179. Status status;
  180. if (nn || port) {
  181. if (!port) {
  182. port = kDefaultPort;
  183. }
  184. std::string port_as_string = std::to_string(*port);
  185. status = fs->Connect(nn.value_or(""), port_as_string);
  186. } else {
  187. status = fs->ConnectToDefaultFs();
  188. }
  189. if (!status.ok()) {
  190. Error(status);
  191. // FileSystem's ctor might take ownership of the io_service; if it does,
  192. // it will null out the pointer
  193. if (io_service)
  194. delete io_service;
  195. delete fs;
  196. return nullptr;
  197. }
  198. return new hdfs_internal(fs);
  199. } catch (const std::exception & e) {
  200. ReportException(e);
  201. return nullptr;
  202. } catch (...) {
  203. ReportCaughtNonException();
  204. return nullptr;
  205. }
  206. }
  207. hdfsFS hdfsConnect(const char *nn, tPort port) {
  208. return hdfsConnectAsUser(nn, port, "");
  209. }
  210. hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user) {
  211. return doHdfsConnect(std::string(nn), port, std::string(user), Options());
  212. }
  213. int hdfsDisconnect(hdfsFS fs) {
  214. try
  215. {
  216. if (!fs) {
  217. ReportError(ENODEV, "Cannot disconnect null FS handle.");
  218. return -1;
  219. }
  220. delete fs;
  221. return 0;
  222. } catch (const std::exception & e) {
  223. return ReportException(e);
  224. } catch (...) {
  225. return ReportCaughtNonException();
  226. }
  227. }
  228. hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, int bufferSize,
  229. short replication, tSize blocksize) {
  230. try
  231. {
  232. (void)flags;
  233. (void)bufferSize;
  234. (void)replication;
  235. (void)blocksize;
  236. if (!fs) {
  237. ReportError(ENODEV, "Cannot perform FS operations with null FS handle.");
  238. return nullptr;
  239. }
  240. FileHandle *f = nullptr;
  241. Status stat = fs->get_impl()->Open(path, &f);
  242. if (!stat.ok()) {
  243. Error(stat);
  244. return nullptr;
  245. }
  246. return new hdfsFile_internal(f);
  247. } catch (const std::exception & e) {
  248. ReportException(e);
  249. return nullptr;
  250. } catch (...) {
  251. ReportCaughtNonException();
  252. return nullptr;
  253. }
  254. }
  255. int hdfsCloseFile(hdfsFS fs, hdfsFile file) {
  256. try
  257. {
  258. if (!CheckSystemAndHandle(fs, file)) {
  259. return -1;
  260. }
  261. delete file;
  262. return 0;
  263. } catch (const std::exception & e) {
  264. return ReportException(e);
  265. } catch (...) {
  266. return ReportCaughtNonException();
  267. }
  268. }
  269. tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void *buffer,
  270. tSize length) {
  271. try
  272. {
  273. if (!CheckSystemAndHandle(fs, file)) {
  274. return -1;
  275. }
  276. size_t len = length;
  277. Status stat = file->get_impl()->PositionRead(buffer, &len, position);
  278. if(!stat.ok()) {
  279. return Error(stat);
  280. }
  281. return (tSize)len;
  282. } catch (const std::exception & e) {
  283. return ReportException(e);
  284. } catch (...) {
  285. return ReportCaughtNonException();
  286. }
  287. }
  288. tSize hdfsRead(hdfsFS fs, hdfsFile file, void *buffer, tSize length) {
  289. try
  290. {
  291. if (!CheckSystemAndHandle(fs, file)) {
  292. return -1;
  293. }
  294. size_t len = length;
  295. Status stat = file->get_impl()->Read(buffer, &len);
  296. if (!stat.ok()) {
  297. return Error(stat);
  298. }
  299. return (tSize)len;
  300. } catch (const std::exception & e) {
  301. return ReportException(e);
  302. } catch (...) {
  303. return ReportCaughtNonException();
  304. }
  305. }
  306. /* 0 on success, -1 on error*/
  307. int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos) {
  308. try
  309. {
  310. if (!CheckSystemAndHandle(fs, file)) {
  311. return -1;
  312. }
  313. off_t desired = desiredPos;
  314. Status stat = file->get_impl()->Seek(&desired, std::ios_base::beg);
  315. if (!stat.ok()) {
  316. return Error(stat);
  317. }
  318. return 0;
  319. } catch (const std::exception & e) {
  320. return ReportException(e);
  321. } catch (...) {
  322. return ReportCaughtNonException();
  323. }
  324. }
  325. tOffset hdfsTell(hdfsFS fs, hdfsFile file) {
  326. try
  327. {
  328. if (!CheckSystemAndHandle(fs, file)) {
  329. return -1;
  330. }
  331. ssize_t offset = 0;
  332. Status stat = file->get_impl()->Seek(&offset, std::ios_base::cur);
  333. if (!stat.ok()) {
  334. return Error(stat);
  335. }
  336. return offset;
  337. } catch (const std::exception & e) {
  338. return ReportException(e);
  339. } catch (...) {
  340. return ReportCaughtNonException();
  341. }
  342. }
  343. /* extended API */
  344. int hdfsCancel(hdfsFS fs, hdfsFile file) {
  345. try
  346. {
  347. if (!CheckSystemAndHandle(fs, file)) {
  348. return -1;
  349. }
  350. static_cast<FileHandleImpl*>(file->get_impl())->CancelOperations();
  351. return 0;
  352. } catch (const std::exception & e) {
  353. return ReportException(e);
  354. } catch (...) {
  355. return ReportCaughtNonException();
  356. }
  357. }
  358. /*******************************************************************
  359. * EVENT CALLBACKS
  360. *******************************************************************/
  361. const char * FS_NN_CONNECT_EVENT = hdfs::FS_NN_CONNECT_EVENT;
  362. const char * FS_NN_READ_EVENT = hdfs::FS_NN_READ_EVENT;
  363. const char * FS_NN_WRITE_EVENT = hdfs::FS_NN_WRITE_EVENT;
  364. const char * FILE_DN_CONNECT_EVENT = hdfs::FILE_DN_CONNECT_EVENT;
  365. const char * FILE_DN_READ_EVENT = hdfs::FILE_DN_READ_EVENT;
  366. const char * FILE_DN_WRITE_EVENT = hdfs::FILE_DN_WRITE_EVENT;
  367. event_response fs_callback_glue(libhdfspp_fs_event_callback handler,
  368. int64_t cookie,
  369. const char * event,
  370. const char * cluster,
  371. int64_t value) {
  372. int result = handler(event, cluster, value, cookie);
  373. if (result == LIBHDFSPP_EVENT_OK) {
  374. return event_response::ok();
  375. }
  376. #ifndef NDEBUG
  377. if (result == DEBUG_SIMULATE_ERROR) {
  378. return event_response::test_err(Status::Error("Simulated error"));
  379. }
  380. #endif
  381. return event_response::ok();
  382. }
  383. event_response file_callback_glue(libhdfspp_file_event_callback handler,
  384. int64_t cookie,
  385. const char * event,
  386. const char * cluster,
  387. const char * file,
  388. int64_t value) {
  389. int result = handler(event, cluster, file, value, cookie);
  390. if (result == LIBHDFSPP_EVENT_OK) {
  391. return event_response::ok();
  392. }
  393. #ifndef NDEBUG
  394. if (result == DEBUG_SIMULATE_ERROR) {
  395. return event_response::test_err(Status::Error("Simulated error"));
  396. }
  397. #endif
  398. return event_response::ok();
  399. }
  400. int hdfsPreAttachFSMonitor(libhdfspp_fs_event_callback handler, int64_t cookie)
  401. {
  402. fs_event_callback callback = std::bind(fs_callback_glue, handler, cookie, _1, _2, _3);
  403. fsEventCallback = callback;
  404. return 0;
  405. }
  406. int hdfsPreAttachFileMonitor(libhdfspp_file_event_callback handler, int64_t cookie)
  407. {
  408. file_event_callback callback = std::bind(file_callback_glue, handler, cookie, _1, _2, _3, _4);
  409. fileEventCallback = callback;
  410. return 0;
  411. }
  412. /*******************************************************************
  413. * BUILDER INTERFACE
  414. *******************************************************************/
  415. HdfsConfiguration LoadDefault(ConfigurationLoader & loader)
  416. {
  417. optional<HdfsConfiguration> result = loader.LoadDefaultResources<HdfsConfiguration>();
  418. if (result)
  419. {
  420. return result.value();
  421. }
  422. else
  423. {
  424. return loader.New<HdfsConfiguration>();
  425. }
  426. }
  427. hdfsBuilder::hdfsBuilder() : config(loader.New<HdfsConfiguration>())
  428. {
  429. loader.SetDefaultSearchPath();
  430. config = LoadDefault(loader);
  431. }
  432. hdfsBuilder::hdfsBuilder(const char * directory) :
  433. config(loader.New<HdfsConfiguration>())
  434. {
  435. loader.SetSearchPath(directory);
  436. config = LoadDefault(loader);
  437. }
  438. struct hdfsBuilder *hdfsNewBuilder(void)
  439. {
  440. try
  441. {
  442. return new struct hdfsBuilder();
  443. } catch (const std::exception & e) {
  444. ReportException(e);
  445. return nullptr;
  446. } catch (...) {
  447. ReportCaughtNonException();
  448. return nullptr;
  449. }
  450. }
  451. void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn)
  452. {
  453. bld->overrideHost = std::string(nn);
  454. }
  455. void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port)
  456. {
  457. bld->overridePort = port;
  458. }
  459. void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName)
  460. {
  461. if (userName && *userName) {
  462. bld->user = std::string(userName);
  463. }
  464. }
  465. void hdfsFreeBuilder(struct hdfsBuilder *bld)
  466. {
  467. try
  468. {
  469. delete bld;
  470. } catch (const std::exception & e) {
  471. ReportException(e);
  472. } catch (...) {
  473. ReportCaughtNonException();
  474. }
  475. }
  476. int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
  477. const char *val)
  478. {
  479. try
  480. {
  481. optional<HdfsConfiguration> newConfig = bld->loader.OverlayValue(bld->config, key, val);
  482. if (newConfig)
  483. {
  484. bld->config = newConfig.value();
  485. return 0;
  486. }
  487. else
  488. {
  489. ReportError(EINVAL, "Could not change Builder value");
  490. return 1;
  491. }
  492. } catch (const std::exception & e) {
  493. return ReportException(e);
  494. } catch (...) {
  495. return ReportCaughtNonException();
  496. }
  497. }
  498. void hdfsConfStrFree(char *val)
  499. {
  500. free(val);
  501. }
  502. hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld) {
  503. return doHdfsConnect(bld->overrideHost, bld->overridePort, bld->user, bld->config.GetOptions());
  504. }
  505. int hdfsConfGetStr(const char *key, char **val)
  506. {
  507. try
  508. {
  509. hdfsBuilder builder;
  510. return hdfsBuilderConfGetStr(&builder, key, val);
  511. } catch (const std::exception & e) {
  512. return ReportException(e);
  513. } catch (...) {
  514. return ReportCaughtNonException();
  515. }
  516. }
  517. int hdfsConfGetInt(const char *key, int32_t *val)
  518. {
  519. try
  520. {
  521. hdfsBuilder builder;
  522. return hdfsBuilderConfGetInt(&builder, key, val);
  523. } catch (const std::exception & e) {
  524. return ReportException(e);
  525. } catch (...) {
  526. return ReportCaughtNonException();
  527. }
  528. }
  529. //
  530. // Extended builder interface
  531. //
  532. struct hdfsBuilder *hdfsNewBuilderFromDirectory(const char * configDirectory)
  533. {
  534. try
  535. {
  536. return new struct hdfsBuilder(configDirectory);
  537. } catch (const std::exception & e) {
  538. ReportException(e);
  539. return nullptr;
  540. } catch (...) {
  541. ReportCaughtNonException();
  542. return nullptr;
  543. }
  544. }
  545. int hdfsBuilderConfGetStr(struct hdfsBuilder *bld, const char *key,
  546. char **val)
  547. {
  548. try
  549. {
  550. optional<std::string> value = bld->config.Get(key);
  551. if (value)
  552. {
  553. size_t len = value->length() + 1;
  554. *val = static_cast<char *>(malloc(len));
  555. strncpy(*val, value->c_str(), len);
  556. }
  557. else
  558. {
  559. *val = nullptr;
  560. }
  561. return 0;
  562. } catch (const std::exception & e) {
  563. return ReportException(e);
  564. } catch (...) {
  565. return ReportCaughtNonException();
  566. }
  567. }
  568. // If we're running on a 32-bit platform, we might get 64-bit values that
  569. // don't fit in an int, and int is specified by the java hdfs.h interface
  570. bool isValidInt(int64_t value)
  571. {
  572. return (value >= std::numeric_limits<int>::min() &&
  573. value <= std::numeric_limits<int>::max());
  574. }
  575. int hdfsBuilderConfGetInt(struct hdfsBuilder *bld, const char *key, int32_t *val)
  576. {
  577. try
  578. {
  579. // Pull from default configuration
  580. optional<int64_t> value = bld->config.GetInt(key);
  581. if (value)
  582. {
  583. if (!isValidInt(*value))
  584. return 1;
  585. *val = *value;
  586. }
  587. // If not found, don't change val
  588. ReportError(EINVAL, "Could not get Builder value");
  589. return 0;
  590. } catch (const std::exception & e) {
  591. return ReportException(e);
  592. } catch (...) {
  593. return ReportCaughtNonException();
  594. }
  595. }
  596. /**
  597. * Logging functions
  598. **/
  599. class CForwardingLogger : public LoggerInterface {
  600. public:
  601. CForwardingLogger() : callback_(nullptr) {};
  602. // Converts LogMessage into LogData, a POD type,
  603. // and invokes callback_ if it's not null.
  604. void Write(const LogMessage& msg);
  605. // pass in NULL to clear the hook
  606. void SetCallback(void (*callback)(LogData*));
  607. //return a copy, or null on failure.
  608. static LogData *CopyLogData(const LogData*);
  609. //free LogData allocated with CopyLogData
  610. static void FreeLogData(LogData*);
  611. private:
  612. void (*callback_)(LogData*);
  613. };
  614. /**
  615. * Plugin to forward message to a C function pointer
  616. **/
  617. void CForwardingLogger::Write(const LogMessage& msg) {
  618. if(!callback_)
  619. return;
  620. const std::string text = msg.MsgString();
  621. LogData data;
  622. data.level = msg.level();
  623. data.component = msg.component();
  624. data.msg = text.c_str();
  625. data.file_name = msg.file_name();
  626. data.file_line = msg.file_line();
  627. callback_(&data);
  628. }
  629. void CForwardingLogger::SetCallback(void (*callback)(LogData*)) {
  630. callback_ = callback;
  631. }
  632. LogData *CForwardingLogger::CopyLogData(const LogData *orig) {
  633. if(!orig)
  634. return nullptr;
  635. LogData *copy = (LogData*)malloc(sizeof(LogData));
  636. if(!copy)
  637. return nullptr;
  638. copy->level = orig->level;
  639. copy->component = orig->component;
  640. if(orig->msg)
  641. copy->msg = strdup(orig->msg);
  642. copy->file_name = orig->file_name;
  643. copy->file_line = orig->file_line;
  644. return copy;
  645. }
  646. void CForwardingLogger::FreeLogData(LogData *data) {
  647. if(!data)
  648. return;
  649. if(data->msg)
  650. free((void*)data->msg);
  651. // Inexpensive way to help catch use-after-free
  652. memset(data, 0, sizeof(LogData));
  653. free(data);
  654. }
  655. LogData *hdfsCopyLogData(LogData *data) {
  656. return CForwardingLogger::CopyLogData(data);
  657. }
  658. void hdfsFreeLogData(LogData *data) {
  659. CForwardingLogger::FreeLogData(data);
  660. }
  661. void hdfsSetLogFunction(void (*callback)(LogData*)) {
  662. CForwardingLogger *logger = new CForwardingLogger();
  663. logger->SetCallback(callback);
  664. LogManager::SetLoggerImplementation(std::unique_ptr<LoggerInterface>(logger));
  665. }
  666. static bool IsLevelValid(int component) {
  667. if(component < HDFSPP_LOG_LEVEL_TRACE || component > HDFSPP_LOG_LEVEL_ERROR)
  668. return false;
  669. return true;
  670. }
  671. // should use __builtin_popcnt as optimization on some platforms
  672. static int popcnt(int val) {
  673. int bits = sizeof(val) * 8;
  674. int count = 0;
  675. for(int i=0; i<bits; i++) {
  676. if((val >> i) & 0x1)
  677. count++;
  678. }
  679. return count;
  680. }
  681. static bool IsComponentValid(int component) {
  682. if(component < HDFSPP_LOG_COMPONENT_UNKNOWN || component > HDFSPP_LOG_COMPONENT_FILESYSTEM)
  683. return false;
  684. if(popcnt(component) != 1)
  685. return false;
  686. return true;
  687. }
  688. int hdfsEnableLoggingForComponent(int component) {
  689. if(!IsComponentValid(component))
  690. return 1;
  691. LogManager::EnableLogForComponent(static_cast<LogSourceComponent>(component));
  692. return 0;
  693. }
  694. int hdfsDisableLoggingForComponent(int component) {
  695. if(!IsComponentValid(component))
  696. return 1;
  697. LogManager::DisableLogForComponent(static_cast<LogSourceComponent>(component));
  698. return 0;
  699. }
  700. int hdfsSetLoggingLevel(int level) {
  701. if(!IsLevelValid(level))
  702. return 1;
  703. LogManager::SetLogLevel(static_cast<LogLevel>(level));
  704. return 0;
  705. }