hdfs_ext.h 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394
  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. #ifndef LIBHDFSPP_HDFS_HDFSEXT
  19. #define LIBHDFSPP_HDFS_HDFSEXT
  20. #include <hdfspp/log.h>
  21. /* get typdefs and #defines from libhdfs' hdfs.h to stay consistent */
  22. #include <hdfs/hdfs.h>
  23. /**
  24. * Note: The #defines below are copied directly from libhdfs'
  25. * hdfs.h. LIBHDFS_EXTERNAL gets explicitly #undefed at the
  26. * end of the file so it must be redefined here.
  27. **/
  28. #ifdef WIN32
  29. #ifdef LIBHDFS_DLL_EXPORT
  30. #define LIBHDFS_EXTERNAL __declspec(dllexport)
  31. #elif LIBHDFS_DLL_IMPORT
  32. #define LIBHDFS_EXTERNAL __declspec(dllimport)
  33. #else
  34. #define LIBHDFS_EXTERNAL
  35. #endif
  36. #else
  37. #ifdef LIBHDFS_DLL_EXPORT
  38. #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
  39. #elif LIBHDFS_DLL_IMPORT
  40. #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
  41. #else
  42. #define LIBHDFS_EXTERNAL
  43. #endif
  44. #endif
  45. /**
  46. * Keep C bindings that are libhdfs++ specific in here.
  47. **/
  48. #ifdef __cplusplus
  49. extern "C" {
  50. #endif
  51. /**
  52. * Reads the last error, if any, that happened in this thread
  53. * into the user supplied buffer.
  54. * @param buf A chunk of memory with room for the error string.
  55. * @param len Size of the buffer, if the message is longer than
  56. * len len-1 bytes of the message will be copied.
  57. * @return 0 on successful read of the last error, -1 otherwise.
  58. **/
  59. LIBHDFS_EXTERNAL
  60. int hdfsGetLastError(char *buf, int len);
  61. /**
  62. * Cancels operations being made by the FileHandle.
  63. * Note: Cancel cannot be reversed. This is intended
  64. * to be used before hdfsClose to avoid waiting for
  65. * operations to complete.
  66. **/
  67. LIBHDFS_EXTERNAL
  68. int hdfsCancel(hdfsFS fs, hdfsFile file);
  69. /**
  70. * Create an HDFS builder, using the configuration XML files from the indicated
  71. * directory. If the directory does not exist, or contains no configuration
  72. * XML files, a Builder using all default values will be returned.
  73. *
  74. * @return The HDFS builder, or NULL on error.
  75. */
  76. struct hdfsBuilder *hdfsNewBuilderFromDirectory(const char * configDirectory);
  77. /**
  78. * Get a configuration string from the settings currently read into the builder.
  79. *
  80. * @param key The key to find
  81. * @param val (out param) The value. This will be set to NULL if the
  82. * key isn't found. You must free this string with
  83. * hdfsConfStrFree.
  84. *
  85. * @return 0 on success; -1 otherwise.
  86. * Failure to find the key is not an error.
  87. */
  88. LIBHDFS_EXTERNAL
  89. int hdfsBuilderConfGetStr(struct hdfsBuilder *bld, const char *key,
  90. char **val);
  91. /**
  92. * Get a configuration integer from the settings currently read into the builder.
  93. *
  94. * @param key The key to find
  95. * @param val (out param) The value. This will NOT be changed if the
  96. * key isn't found.
  97. *
  98. * @return 0 on success; -1 otherwise.
  99. * Failure to find the key is not an error.
  100. */
  101. LIBHDFS_EXTERNAL
  102. int hdfsBuilderConfGetInt(struct hdfsBuilder *bld, const char *key, int32_t *val);
  103. /**
  104. * Get a configuration long from the settings currently read into the builder.
  105. *
  106. * @param key The key to find
  107. * @param val (out param) The value. This will NOT be changed if the
  108. * key isn't found.
  109. *
  110. * @return 0 on success; -1 otherwise.
  111. * Failure to find the key is not an error.
  112. */
  113. LIBHDFS_EXTERNAL
  114. int hdfsBuilderConfGetLong(struct hdfsBuilder *bld, const char *key, int64_t *val);
  115. struct hdfsDNInfo {
  116. const char * ip_address;
  117. const char * hostname;
  118. const char * network_location;
  119. int xfer_port;
  120. int info_port;
  121. int IPC_port;
  122. int info_secure_port;
  123. };
  124. struct hdfsBlockInfo {
  125. uint64_t start_offset;
  126. uint64_t num_bytes;
  127. size_t num_locations;
  128. struct hdfsDNInfo * locations;
  129. };
  130. struct hdfsBlockLocations
  131. {
  132. uint64_t fileLength;
  133. int isLastBlockComplete;
  134. int isUnderConstruction;
  135. size_t num_blocks;
  136. struct hdfsBlockInfo * blocks;
  137. };
  138. /**
  139. * Returns the block information and data nodes associated with a particular file.
  140. *
  141. * The hdfsBlockLocations structure will have zero or more hdfsBlockInfo elements,
  142. * which will have zero or more ip_addr elements indicating which datanodes have
  143. * each block.
  144. *
  145. * @param fs A connected hdfs instance
  146. * @param path Path of the file to query
  147. * @param locations The address of an output pointer to contain the block information.
  148. * On success, this pointer must be later freed with hdfsFreeBlockLocations.
  149. *
  150. * @return 0 on success; -1 otherwise.
  151. * If the file does not exist, -1 will be returned and errno will be set.
  152. */
  153. LIBHDFS_EXTERNAL
  154. int hdfsGetBlockLocations(hdfsFS fs, const char *path, struct hdfsBlockLocations ** locations);
  155. /**
  156. * Frees up an hdfsBlockLocations pointer allocated by hdfsGetBlockLocations.
  157. *
  158. * @param locations The previously-populated pointer allocated by hdfsGetBlockLocations
  159. * @return 0 on success, -1 on error
  160. */
  161. LIBHDFS_EXTERNAL
  162. int hdfsFreeBlockLocations(struct hdfsBlockLocations * locations);
  163. /**
  164. * Client can supply a C style function pointer to be invoked any time something
  165. * is logged. Unlike the C++ logger this will not filter by level or component,
  166. * it is up to the consumer to throw away messages they don't want.
  167. *
  168. * Note: The callback provided must be reentrant, the library does not guarentee
  169. * that there won't be concurrent calls.
  170. * Note: Callback does not own the LogData struct. If the client would like to
  171. * keep one around use hdfsCopyLogData/hdfsFreeLogData.
  172. **/
  173. LIBHDFS_EXTERNAL
  174. void hdfsSetLogFunction(void (*hook)(LogData*));
  175. /**
  176. * Create a copy of the LogData object passed in and return a pointer to it.
  177. * Returns null if it was unable to copy/
  178. **/
  179. LIBHDFS_EXTERNAL
  180. LogData *hdfsCopyLogData(const LogData*);
  181. /**
  182. * Client must call this to dispose of the LogData created by hdfsCopyLogData.
  183. **/
  184. LIBHDFS_EXTERNAL
  185. void hdfsFreeLogData(LogData*);
  186. /**
  187. * Enable loggind functionality for a component.
  188. * Return -1 on failure, 0 otherwise.
  189. **/
  190. LIBHDFS_EXTERNAL
  191. int hdfsEnableLoggingForComponent(int component);
  192. /**
  193. * Disable logging functionality for a component.
  194. * Return -1 on failure, 0 otherwise.
  195. **/
  196. LIBHDFS_EXTERNAL
  197. int hdfsDisableLoggingForComponent(int component);
  198. /**
  199. * Set level between trace and error.
  200. * Return -1 on failure, 0 otherwise.
  201. **/
  202. LIBHDFS_EXTERNAL
  203. int hdfsSetLoggingLevel(int component);
  204. /*
  205. * Supported event names. These names will stay consistent in libhdfs callbacks.
  206. *
  207. * Other events not listed here may be seen, but they are not stable and
  208. * should not be counted on.
  209. */
  210. extern const char * FS_NN_CONNECT_EVENT;
  211. extern const char * FS_NN_READ_EVENT;
  212. extern const char * FS_NN_WRITE_EVENT;
  213. extern const char * FILE_DN_CONNECT_EVENT;
  214. extern const char * FILE_DN_READ_EVENT;
  215. extern const char * FILE_DN_WRITE_EVENT;
  216. #define LIBHDFSPP_EVENT_OK (0)
  217. #define DEBUG_SIMULATE_ERROR (-1)
  218. typedef int (*libhdfspp_fs_event_callback)(const char * event, const char * cluster,
  219. int64_t value, int64_t cookie);
  220. typedef int (*libhdfspp_file_event_callback)(const char * event,
  221. const char * cluster,
  222. const char * file,
  223. int64_t value, int64_t cookie);
  224. /**
  225. * Registers a callback for the next filesystem connect operation the current
  226. * thread executes.
  227. *
  228. * @param handler A function pointer. Taken as a void* and internally
  229. * cast into the appropriate type.
  230. * @param cookie An opaque value that will be passed into the handler; can
  231. * be used to correlate the handler with some object in the
  232. * consumer's space.
  233. **/
  234. LIBHDFS_EXTERNAL
  235. int hdfsPreAttachFSMonitor(libhdfspp_fs_event_callback handler, int64_t cookie);
  236. /**
  237. * Registers a callback for the next file open operation the current thread
  238. * executes.
  239. *
  240. * @param fs The filesystem
  241. * @param handler A function pointer. Taken as a void* and internally
  242. * cast into the appropriate type.
  243. * @param cookie An opaque value that will be passed into the handler; can
  244. * be used to correlate the handler with some object in the
  245. * consumer's space.
  246. **/
  247. LIBHDFS_EXTERNAL
  248. int hdfsPreAttachFileMonitor(libhdfspp_file_event_callback handler, int64_t cookie);
  249. /**
  250. * Finds file name on the file system. hdfsFreeFileInfo should be called to deallocate memory.
  251. *
  252. * @param fs The filesystem (required)
  253. * @param path Path at which to begin search, can have wild cards (must be non-blank)
  254. * @param name Name to find, can have wild cards (must be non-blank)
  255. * @param numEntries Set to the number of files/directories in the result.
  256. * @return Returns a dynamically-allocated array of hdfsFileInfo
  257. * objects; NULL on error or empty result.
  258. * errno is set to non-zero on error or zero on success.
  259. **/
  260. LIBHDFS_EXTERNAL
  261. hdfsFileInfo * hdfsFind(hdfsFS fs, const char* path, const char* name, uint32_t * numEntries);
  262. /*****************************************************************************
  263. * HDFS SNAPSHOT FUNCTIONS
  264. ****************************************************************************/
  265. /**
  266. * Creates a snapshot of a snapshottable directory specified by path
  267. *
  268. * @param fs The filesystem (required)
  269. * @param path Path to the directory to be snapshotted (must be non-blank)
  270. * @param name Name to be given to the created snapshot (may be NULL)
  271. * @return 0 on success, corresponding errno on failure
  272. **/
  273. LIBHDFS_EXTERNAL
  274. int hdfsCreateSnapshot(hdfsFS fs, const char* path, const char* name);
  275. /**
  276. * Deletes the directory snapshot specified by path and name
  277. *
  278. * @param fs The filesystem (required)
  279. * @param path Path to the snapshotted directory (must be non-blank)
  280. * @param name Name of the snapshot to be deleted (must be non-blank)
  281. * @return 0 on success, corresponding errno on failure
  282. **/
  283. LIBHDFS_EXTERNAL
  284. int hdfsDeleteSnapshot(hdfsFS fs, const char* path, const char* name);
  285. /**
  286. * Renames the directory snapshot specified by path from old_name to new_name
  287. *
  288. * @param fs The filesystem (required)
  289. * @param path Path to the snapshotted directory (must be non-blank)
  290. * @param old_name Current name of the snapshot (must be non-blank)
  291. * @param new_name New name of the snapshot (must be non-blank)
  292. * @return 0 on success, corresponding errno on failure
  293. **/
  294. int hdfsRenameSnapshot(hdfsFS fs, const char* path, const char* old_name, const char* new_name);
  295. /**
  296. * Allows snapshots to be made on the specified directory
  297. *
  298. * @param fs The filesystem (required)
  299. * @param path Path to the directory to be made snapshottable (must be non-blank)
  300. * @return 0 on success, corresponding errno on failure
  301. **/
  302. LIBHDFS_EXTERNAL
  303. int hdfsAllowSnapshot(hdfsFS fs, const char* path);
  304. /**
  305. * Disallows snapshots to be made on the specified directory
  306. *
  307. * @param fs The filesystem (required)
  308. * @param path Path to the directory to be made non-snapshottable (must be non-blank)
  309. * @return 0 on success, corresponding errno on failure
  310. **/
  311. LIBHDFS_EXTERNAL
  312. int hdfsDisallowSnapshot(hdfsFS fs, const char* path);
  313. /**
  314. * Create a FileSystem based on the builder but don't connect
  315. * @param bld Used to populate config options in the same manner as hdfsBuilderConnect.
  316. * Does not free builder.
  317. **/
  318. LIBHDFS_EXTERNAL
  319. hdfsFS hdfsAllocateFileSystem(struct hdfsBuilder *bld);
  320. /**
  321. * Connect a FileSystem created with hdfsAllocateFileSystem
  322. * @param fs A disconnected FS created with hdfsAllocateFileSystem
  323. * @param bld The same or exact copy of the builder used for Allocate, we still need a few fields.
  324. * Does not free builder.
  325. * @return 0 on success, corresponding errno on failure
  326. **/
  327. LIBHDFS_EXTERNAL
  328. int hdfsConnectAllocated(hdfsFS fs, struct hdfsBuilder *bld);
  329. /**
  330. * Cancel a pending connection on a FileSystem
  331. * @param fs A fs in the process of connecting using hdfsConnectAllocated in another thread.
  332. * @return 0 on success, corresponding errno on failure
  333. **/
  334. LIBHDFS_EXTERNAL
  335. int hdfsCancelPendingConnection(hdfsFS fs);
  336. #ifdef __cplusplus
  337. } /* end extern "C" */
  338. #endif
  339. #endif