hdfs.h 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799
  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 LIBHDFS_HDFS_H
  19. #define LIBHDFS_HDFS_H
  20. #include <errno.h> /* for EINTERNAL, etc. */
  21. #include <fcntl.h> /* for O_RDONLY, O_WRONLY */
  22. #include <stdint.h> /* for uint64_t, etc. */
  23. #include <time.h> /* for time_t */
  24. #ifndef O_RDONLY
  25. #define O_RDONLY 1
  26. #endif
  27. #ifndef O_WRONLY
  28. #define O_WRONLY 2
  29. #endif
  30. #ifndef EINTERNAL
  31. #define EINTERNAL 255
  32. #endif
  33. #define ELASTIC_BYTE_BUFFER_POOL_CLASS \
  34. "org/apache/hadoop/io/ElasticByteBufferPool"
  35. /** All APIs set errno to meaningful values */
  36. #ifdef __cplusplus
  37. extern "C" {
  38. #endif
  39. /**
  40. * Some utility decls used in libhdfs.
  41. */
  42. struct hdfsBuilder;
  43. typedef int32_t tSize; /// size of data for read/write io ops
  44. typedef time_t tTime; /// time type in seconds
  45. typedef int64_t tOffset;/// offset within the file
  46. typedef uint16_t tPort; /// port
  47. typedef enum tObjectKind {
  48. kObjectKindFile = 'F',
  49. kObjectKindDirectory = 'D',
  50. } tObjectKind;
  51. /**
  52. * The C reflection of org.apache.org.hadoop.FileSystem .
  53. */
  54. struct hdfs_internal;
  55. typedef struct hdfs_internal* hdfsFS;
  56. struct hdfsFile_internal;
  57. typedef struct hdfsFile_internal* hdfsFile;
  58. struct hadoopRzOptions;
  59. struct hadoopRzBuffer;
  60. /**
  61. * Determine if a file is open for read.
  62. *
  63. * @param file The HDFS file
  64. * @return 1 if the file is open for read; 0 otherwise
  65. */
  66. int hdfsFileIsOpenForRead(hdfsFile file);
  67. /**
  68. * Determine if a file is open for write.
  69. *
  70. * @param file The HDFS file
  71. * @return 1 if the file is open for write; 0 otherwise
  72. */
  73. int hdfsFileIsOpenForWrite(hdfsFile file);
  74. struct hdfsReadStatistics {
  75. uint64_t totalBytesRead;
  76. uint64_t totalLocalBytesRead;
  77. uint64_t totalShortCircuitBytesRead;
  78. uint64_t totalZeroCopyBytesRead;
  79. };
  80. /**
  81. * Get read statistics about a file. This is only applicable to files
  82. * opened for reading.
  83. *
  84. * @param file The HDFS file
  85. * @param stats (out parameter) on a successful return, the read
  86. * statistics. Unchanged otherwise. You must free the
  87. * returned statistics with hdfsFileFreeReadStatistics.
  88. * @return 0 if the statistics were successfully returned,
  89. * -1 otherwise. On a failure, please check errno against
  90. * ENOTSUP. webhdfs, LocalFilesystem, and so forth may
  91. * not support read statistics.
  92. */
  93. int hdfsFileGetReadStatistics(hdfsFile file,
  94. struct hdfsReadStatistics **stats);
  95. /**
  96. * @param stats HDFS read statistics for a file.
  97. *
  98. * @return the number of remote bytes read.
  99. */
  100. int64_t hdfsReadStatisticsGetRemoteBytesRead(
  101. const struct hdfsReadStatistics *stats);
  102. /**
  103. * Free some HDFS read statistics.
  104. *
  105. * @param stats The HDFS read statistics to free.
  106. */
  107. void hdfsFileFreeReadStatistics(struct hdfsReadStatistics *stats);
  108. /**
  109. * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
  110. * Connect to the hdfs.
  111. * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
  112. * @param port The port on which the server is listening.
  113. * @param user the user name (this is hadoop domain user). Or NULL is equivelant to hhdfsConnect(host, port)
  114. * @return Returns a handle to the filesystem or NULL on error.
  115. * @deprecated Use hdfsBuilderConnect instead.
  116. */
  117. hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user);
  118. /**
  119. * hdfsConnect - Connect to a hdfs file system.
  120. * Connect to the hdfs.
  121. * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
  122. * @param port The port on which the server is listening.
  123. * @return Returns a handle to the filesystem or NULL on error.
  124. * @deprecated Use hdfsBuilderConnect instead.
  125. */
  126. hdfsFS hdfsConnect(const char* nn, tPort port);
  127. /**
  128. * hdfsConnect - Connect to an hdfs file system.
  129. *
  130. * Forces a new instance to be created
  131. *
  132. * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
  133. * @param port The port on which the server is listening.
  134. * @param user The user name to use when connecting
  135. * @return Returns a handle to the filesystem or NULL on error.
  136. * @deprecated Use hdfsBuilderConnect instead.
  137. */
  138. hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user );
  139. /**
  140. * hdfsConnect - Connect to an hdfs file system.
  141. *
  142. * Forces a new instance to be created
  143. *
  144. * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
  145. * @param port The port on which the server is listening.
  146. * @return Returns a handle to the filesystem or NULL on error.
  147. * @deprecated Use hdfsBuilderConnect instead.
  148. */
  149. hdfsFS hdfsConnectNewInstance(const char* nn, tPort port);
  150. /**
  151. * Connect to HDFS using the parameters defined by the builder.
  152. *
  153. * The HDFS builder will be freed, whether or not the connection was
  154. * successful.
  155. *
  156. * Every successful call to hdfsBuilderConnect should be matched with a call
  157. * to hdfsDisconnect, when the hdfsFS is no longer needed.
  158. *
  159. * @param bld The HDFS builder
  160. * @return Returns a handle to the filesystem, or NULL on error.
  161. */
  162. hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld);
  163. /**
  164. * Create an HDFS builder.
  165. *
  166. * @return The HDFS builder, or NULL on error.
  167. */
  168. struct hdfsBuilder *hdfsNewBuilder(void);
  169. /**
  170. * Force the builder to always create a new instance of the FileSystem,
  171. * rather than possibly finding one in the cache.
  172. *
  173. * @param bld The HDFS builder
  174. */
  175. void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld);
  176. /**
  177. * Set the HDFS NameNode to connect to.
  178. *
  179. * @param bld The HDFS builder
  180. * @param nn The NameNode to use.
  181. *
  182. * If the string given is 'default', the default NameNode
  183. * configuration will be used (from the XML configuration files)
  184. *
  185. * If NULL is given, a LocalFileSystem will be created.
  186. *
  187. * If the string starts with a protocol type such as file:// or
  188. * hdfs://, this protocol type will be used. If not, the
  189. * hdfs:// protocol type will be used.
  190. *
  191. * You may specify a NameNode port in the usual way by
  192. * passing a string of the format hdfs://<hostname>:<port>.
  193. * Alternately, you may set the port with
  194. * hdfsBuilderSetNameNodePort. However, you must not pass the
  195. * port in two different ways.
  196. */
  197. void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn);
  198. /**
  199. * Set the port of the HDFS NameNode to connect to.
  200. *
  201. * @param bld The HDFS builder
  202. * @param port The port.
  203. */
  204. void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port);
  205. /**
  206. * Set the username to use when connecting to the HDFS cluster.
  207. *
  208. * @param bld The HDFS builder
  209. * @param userName The user name. The string will be shallow-copied.
  210. */
  211. void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName);
  212. /**
  213. * Set the path to the Kerberos ticket cache to use when connecting to
  214. * the HDFS cluster.
  215. *
  216. * @param bld The HDFS builder
  217. * @param kerbTicketCachePath The Kerberos ticket cache path. The string
  218. * will be shallow-copied.
  219. */
  220. void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
  221. const char *kerbTicketCachePath);
  222. /**
  223. * Free an HDFS builder.
  224. *
  225. * It is normally not necessary to call this function since
  226. * hdfsBuilderConnect frees the builder.
  227. *
  228. * @param bld The HDFS builder
  229. */
  230. void hdfsFreeBuilder(struct hdfsBuilder *bld);
  231. /**
  232. * Set a configuration string for an HdfsBuilder.
  233. *
  234. * @param key The key to set.
  235. * @param val The value, or NULL to set no value.
  236. * This will be shallow-copied. You are responsible for
  237. * ensuring that it remains valid until the builder is
  238. * freed.
  239. *
  240. * @return 0 on success; nonzero error code otherwise.
  241. */
  242. int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
  243. const char *val);
  244. /**
  245. * Get a configuration string.
  246. *
  247. * @param key The key to find
  248. * @param val (out param) The value. This will be set to NULL if the
  249. * key isn't found. You must free this string with
  250. * hdfsConfStrFree.
  251. *
  252. * @return 0 on success; nonzero error code otherwise.
  253. * Failure to find the key is not an error.
  254. */
  255. int hdfsConfGetStr(const char *key, char **val);
  256. /**
  257. * Get a configuration integer.
  258. *
  259. * @param key The key to find
  260. * @param val (out param) The value. This will NOT be changed if the
  261. * key isn't found.
  262. *
  263. * @return 0 on success; nonzero error code otherwise.
  264. * Failure to find the key is not an error.
  265. */
  266. int hdfsConfGetInt(const char *key, int32_t *val);
  267. /**
  268. * Free a configuration string found with hdfsConfGetStr.
  269. *
  270. * @param val A configuration string obtained from hdfsConfGetStr
  271. */
  272. void hdfsConfStrFree(char *val);
  273. /**
  274. * hdfsDisconnect - Disconnect from the hdfs file system.
  275. * Disconnect from hdfs.
  276. * @param fs The configured filesystem handle.
  277. * @return Returns 0 on success, -1 on error.
  278. * Even if there is an error, the resources associated with the
  279. * hdfsFS will be freed.
  280. */
  281. int hdfsDisconnect(hdfsFS fs);
  282. /**
  283. * hdfsOpenFile - Open a hdfs file in given mode.
  284. * @param fs The configured filesystem handle.
  285. * @param path The full path to the file.
  286. * @param flags - an | of bits/fcntl.h file flags - supported flags are O_RDONLY, O_WRONLY (meaning create or overwrite i.e., implies O_TRUNCAT),
  287. * O_WRONLY|O_APPEND. Other flags are generally ignored other than (O_RDWR || (O_EXCL & O_CREAT)) which return NULL and set errno equal ENOTSUP.
  288. * @param bufferSize Size of buffer for read/write - pass 0 if you want
  289. * to use the default configured values.
  290. * @param replication Block replication - pass 0 if you want to use
  291. * the default configured values.
  292. * @param blocksize Size of block - pass 0 if you want to use the
  293. * default configured values.
  294. * @return Returns the handle to the open file or NULL on error.
  295. */
  296. hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
  297. int bufferSize, short replication, tSize blocksize);
  298. /**
  299. * hdfsCloseFile - Close an open file.
  300. * @param fs The configured filesystem handle.
  301. * @param file The file handle.
  302. * @return Returns 0 on success, -1 on error.
  303. * On error, errno will be set appropriately.
  304. * If the hdfs file was valid, the memory associated with it will
  305. * be freed at the end of this call, even if there was an I/O
  306. * error.
  307. */
  308. int hdfsCloseFile(hdfsFS fs, hdfsFile file);
  309. /**
  310. * hdfsExists - Checks if a given path exsits on the filesystem
  311. * @param fs The configured filesystem handle.
  312. * @param path The path to look for
  313. * @return Returns 0 on success, -1 on error.
  314. */
  315. int hdfsExists(hdfsFS fs, const char *path);
  316. /**
  317. * hdfsSeek - Seek to given offset in file.
  318. * This works only for files opened in read-only mode.
  319. * @param fs The configured filesystem handle.
  320. * @param file The file handle.
  321. * @param desiredPos Offset into the file to seek into.
  322. * @return Returns 0 on success, -1 on error.
  323. */
  324. int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos);
  325. /**
  326. * hdfsTell - Get the current offset in the file, in bytes.
  327. * @param fs The configured filesystem handle.
  328. * @param file The file handle.
  329. * @return Current offset, -1 on error.
  330. */
  331. tOffset hdfsTell(hdfsFS fs, hdfsFile file);
  332. /**
  333. * hdfsRead - Read data from an open file.
  334. * @param fs The configured filesystem handle.
  335. * @param file The file handle.
  336. * @param buffer The buffer to copy read bytes into.
  337. * @param length The length of the buffer.
  338. * @return On success, a positive number indicating how many bytes
  339. * were read.
  340. * On end-of-file, 0.
  341. * On error, -1. Errno will be set to the error code.
  342. * Just like the POSIX read function, hdfsRead will return -1
  343. * and set errno to EINTR if data is temporarily unavailable,
  344. * but we are not yet at the end of the file.
  345. */
  346. tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length);
  347. /**
  348. * hdfsPread - Positional read of data from an open file.
  349. * @param fs The configured filesystem handle.
  350. * @param file The file handle.
  351. * @param position Position from which to read
  352. * @param buffer The buffer to copy read bytes into.
  353. * @param length The length of the buffer.
  354. * @return See hdfsRead
  355. */
  356. tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
  357. void* buffer, tSize length);
  358. /**
  359. * hdfsWrite - Write data into an open file.
  360. * @param fs The configured filesystem handle.
  361. * @param file The file handle.
  362. * @param buffer The data.
  363. * @param length The no. of bytes to write.
  364. * @return Returns the number of bytes written, -1 on error.
  365. */
  366. tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
  367. tSize length);
  368. /**
  369. * hdfsWrite - Flush the data.
  370. * @param fs The configured filesystem handle.
  371. * @param file The file handle.
  372. * @return Returns 0 on success, -1 on error.
  373. */
  374. int hdfsFlush(hdfsFS fs, hdfsFile file);
  375. /**
  376. * hdfsHFlush - Flush out the data in client's user buffer. After the
  377. * return of this call, new readers will see the data.
  378. * @param fs configured filesystem handle
  379. * @param file file handle
  380. * @return 0 on success, -1 on error and sets errno
  381. */
  382. int hdfsHFlush(hdfsFS fs, hdfsFile file);
  383. /**
  384. * hdfsHSync - Similar to posix fsync, Flush out the data in client's
  385. * user buffer. all the way to the disk device (but the disk may have
  386. * it in its cache).
  387. * @param fs configured filesystem handle
  388. * @param file file handle
  389. * @return 0 on success, -1 on error and sets errno
  390. */
  391. int hdfsHSync(hdfsFS fs, hdfsFile file);
  392. /**
  393. * hdfsAvailable - Number of bytes that can be read from this
  394. * input stream without blocking.
  395. * @param fs The configured filesystem handle.
  396. * @param file The file handle.
  397. * @return Returns available bytes; -1 on error.
  398. */
  399. int hdfsAvailable(hdfsFS fs, hdfsFile file);
  400. /**
  401. * hdfsCopy - Copy file from one filesystem to another.
  402. * @param srcFS The handle to source filesystem.
  403. * @param src The path of source file.
  404. * @param dstFS The handle to destination filesystem.
  405. * @param dst The path of destination file.
  406. * @return Returns 0 on success, -1 on error.
  407. */
  408. int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  409. /**
  410. * hdfsMove - Move file from one filesystem to another.
  411. * @param srcFS The handle to source filesystem.
  412. * @param src The path of source file.
  413. * @param dstFS The handle to destination filesystem.
  414. * @param dst The path of destination file.
  415. * @return Returns 0 on success, -1 on error.
  416. */
  417. int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  418. /**
  419. * hdfsDelete - Delete file.
  420. * @param fs The configured filesystem handle.
  421. * @param path The path of the file.
  422. * @param recursive if path is a directory and set to
  423. * non-zero, the directory is deleted else throws an exception. In
  424. * case of a file the recursive argument is irrelevant.
  425. * @return Returns 0 on success, -1 on error.
  426. */
  427. int hdfsDelete(hdfsFS fs, const char* path, int recursive);
  428. /**
  429. * hdfsRename - Rename file.
  430. * @param fs The configured filesystem handle.
  431. * @param oldPath The path of the source file.
  432. * @param newPath The path of the destination file.
  433. * @return Returns 0 on success, -1 on error.
  434. */
  435. int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
  436. /**
  437. * hdfsGetWorkingDirectory - Get the current working directory for
  438. * the given filesystem.
  439. * @param fs The configured filesystem handle.
  440. * @param buffer The user-buffer to copy path of cwd into.
  441. * @param bufferSize The length of user-buffer.
  442. * @return Returns buffer, NULL on error.
  443. */
  444. char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
  445. /**
  446. * hdfsSetWorkingDirectory - Set the working directory. All relative
  447. * paths will be resolved relative to it.
  448. * @param fs The configured filesystem handle.
  449. * @param path The path of the new 'cwd'.
  450. * @return Returns 0 on success, -1 on error.
  451. */
  452. int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
  453. /**
  454. * hdfsCreateDirectory - Make the given file and all non-existent
  455. * parents into directories.
  456. * @param fs The configured filesystem handle.
  457. * @param path The path of the directory.
  458. * @return Returns 0 on success, -1 on error.
  459. */
  460. int hdfsCreateDirectory(hdfsFS fs, const char* path);
  461. /**
  462. * hdfsSetReplication - Set the replication of the specified
  463. * file to the supplied value
  464. * @param fs The configured filesystem handle.
  465. * @param path The path of the file.
  466. * @return Returns 0 on success, -1 on error.
  467. */
  468. int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication);
  469. /**
  470. * hdfsFileInfo - Information about a file/directory.
  471. */
  472. typedef struct {
  473. tObjectKind mKind; /* file or directory */
  474. char *mName; /* the name of the file */
  475. tTime mLastMod; /* the last modification time for the file in seconds */
  476. tOffset mSize; /* the size of the file in bytes */
  477. short mReplication; /* the count of replicas */
  478. tOffset mBlockSize; /* the block size for the file */
  479. char *mOwner; /* the owner of the file */
  480. char *mGroup; /* the group associated with the file */
  481. short mPermissions; /* the permissions associated with the file */
  482. tTime mLastAccess; /* the last access time for the file in seconds */
  483. } hdfsFileInfo;
  484. /**
  485. * hdfsListDirectory - Get list of files/directories for a given
  486. * directory-path. hdfsFreeFileInfo should be called to deallocate memory.
  487. * @param fs The configured filesystem handle.
  488. * @param path The path of the directory.
  489. * @param numEntries Set to the number of files/directories in path.
  490. * @return Returns a dynamically-allocated array of hdfsFileInfo
  491. * objects; NULL on error.
  492. */
  493. hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
  494. int *numEntries);
  495. /**
  496. * hdfsGetPathInfo - Get information about a path as a (dynamically
  497. * allocated) single hdfsFileInfo struct. hdfsFreeFileInfo should be
  498. * called when the pointer is no longer needed.
  499. * @param fs The configured filesystem handle.
  500. * @param path The path of the file.
  501. * @return Returns a dynamically-allocated hdfsFileInfo object;
  502. * NULL on error.
  503. */
  504. hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
  505. /**
  506. * hdfsFreeFileInfo - Free up the hdfsFileInfo array (including fields)
  507. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  508. * objects.
  509. * @param numEntries The size of the array.
  510. */
  511. void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
  512. /**
  513. * hdfsGetHosts - Get hostnames where a particular block (determined by
  514. * pos & blocksize) of a file is stored. The last element in the array
  515. * is NULL. Due to replication, a single block could be present on
  516. * multiple hosts.
  517. * @param fs The configured filesystem handle.
  518. * @param path The path of the file.
  519. * @param start The start of the block.
  520. * @param length The length of the block.
  521. * @return Returns a dynamically-allocated 2-d array of blocks-hosts;
  522. * NULL on error.
  523. */
  524. char*** hdfsGetHosts(hdfsFS fs, const char* path,
  525. tOffset start, tOffset length);
  526. /**
  527. * hdfsFreeHosts - Free up the structure returned by hdfsGetHosts
  528. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  529. * objects.
  530. * @param numEntries The size of the array.
  531. */
  532. void hdfsFreeHosts(char ***blockHosts);
  533. /**
  534. * hdfsGetDefaultBlockSize - Get the default blocksize.
  535. *
  536. * @param fs The configured filesystem handle.
  537. * @deprecated Use hdfsGetDefaultBlockSizeAtPath instead.
  538. *
  539. * @return Returns the default blocksize, or -1 on error.
  540. */
  541. tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
  542. /**
  543. * hdfsGetDefaultBlockSizeAtPath - Get the default blocksize at the
  544. * filesystem indicated by a given path.
  545. *
  546. * @param fs The configured filesystem handle.
  547. * @param path The given path will be used to locate the actual
  548. * filesystem. The full path does not have to exist.
  549. *
  550. * @return Returns the default blocksize, or -1 on error.
  551. */
  552. tOffset hdfsGetDefaultBlockSizeAtPath(hdfsFS fs, const char *path);
  553. /**
  554. * hdfsGetCapacity - Return the raw capacity of the filesystem.
  555. * @param fs The configured filesystem handle.
  556. * @return Returns the raw-capacity; -1 on error.
  557. */
  558. tOffset hdfsGetCapacity(hdfsFS fs);
  559. /**
  560. * hdfsGetUsed - Return the total raw size of all files in the filesystem.
  561. * @param fs The configured filesystem handle.
  562. * @return Returns the total-size; -1 on error.
  563. */
  564. tOffset hdfsGetUsed(hdfsFS fs);
  565. /**
  566. * Change the user and/or group of a file or directory.
  567. *
  568. * @param fs The configured filesystem handle.
  569. * @param path the path to the file or directory
  570. * @param owner User string. Set to NULL for 'no change'
  571. * @param group Group string. Set to NULL for 'no change'
  572. * @return 0 on success else -1
  573. */
  574. int hdfsChown(hdfsFS fs, const char* path, const char *owner,
  575. const char *group);
  576. /**
  577. * hdfsChmod
  578. * @param fs The configured filesystem handle.
  579. * @param path the path to the file or directory
  580. * @param mode the bitmask to set it to
  581. * @return 0 on success else -1
  582. */
  583. int hdfsChmod(hdfsFS fs, const char* path, short mode);
  584. /**
  585. * hdfsUtime
  586. * @param fs The configured filesystem handle.
  587. * @param path the path to the file or directory
  588. * @param mtime new modification time or -1 for no change
  589. * @param atime new access time or -1 for no change
  590. * @return 0 on success else -1
  591. */
  592. int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
  593. /**
  594. * Allocate a zero-copy options structure.
  595. *
  596. * You must free all options structures allocated with this function using
  597. * hadoopRzOptionsFree.
  598. *
  599. * @return A zero-copy options structure, or NULL if one could
  600. * not be allocated. If NULL is returned, errno will
  601. * contain the error number.
  602. */
  603. struct hadoopRzOptions *hadoopRzOptionsAlloc(void);
  604. /**
  605. * Determine whether we should skip checksums in read0.
  606. *
  607. * @param opts The options structure.
  608. * @param skip Nonzero to skip checksums sometimes; zero to always
  609. * check them.
  610. *
  611. * @return 0 on success; -1 plus errno on failure.
  612. */
  613. int hadoopRzOptionsSetSkipChecksum(
  614. struct hadoopRzOptions *opts, int skip);
  615. /**
  616. * Set the ByteBufferPool to use with read0.
  617. *
  618. * @param opts The options structure.
  619. * @param className If this is NULL, we will not use any
  620. * ByteBufferPool. If this is non-NULL, it will be
  621. * treated as the name of the pool class to use.
  622. * For example, you can use
  623. * ELASTIC_BYTE_BUFFER_POOL_CLASS.
  624. *
  625. * @return 0 if the ByteBufferPool class was found and
  626. * instantiated;
  627. * -1 plus errno otherwise.
  628. */
  629. int hadoopRzOptionsSetByteBufferPool(
  630. struct hadoopRzOptions *opts, const char *className);
  631. /**
  632. * Free a hadoopRzOptionsFree structure.
  633. *
  634. * @param opts The options structure to free.
  635. * Any associated ByteBufferPool will also be freed.
  636. */
  637. void hadoopRzOptionsFree(struct hadoopRzOptions *opts);
  638. /**
  639. * Perform a byte buffer read.
  640. * If possible, this will be a zero-copy (mmap) read.
  641. *
  642. * @param file The file to read from.
  643. * @param opts An options structure created by hadoopRzOptionsAlloc.
  644. * @param maxLength The maximum length to read. We may read fewer bytes
  645. * than this length.
  646. *
  647. * @return On success, returns a new hadoopRzBuffer.
  648. * This buffer will continue to be valid and readable
  649. * until it is released by readZeroBufferFree. Failure to
  650. * release a buffer will lead to a memory leak.
  651. *
  652. * NULL plus an errno code on an error.
  653. * errno = EOPNOTSUPP indicates that we could not do a
  654. * zero-copy read, and there was no ByteBufferPool
  655. * supplied.
  656. */
  657. struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
  658. struct hadoopRzOptions *opts, int32_t maxLength);
  659. /**
  660. * Determine the length of the buffer returned from readZero.
  661. *
  662. * @param buffer a buffer returned from readZero.
  663. * @return the length of the buffer.
  664. */
  665. int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer);
  666. /**
  667. * Get a pointer to the raw buffer returned from readZero.
  668. *
  669. * To find out how many bytes this buffer contains, call
  670. * hadoopRzBufferLength.
  671. *
  672. * @param buffer a buffer returned from readZero.
  673. * @return a pointer to the start of the buffer. This will be
  674. * NULL when end-of-file has been reached.
  675. */
  676. const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer);
  677. /**
  678. * Release a buffer obtained through readZero.
  679. *
  680. * @param file The hdfs stream that created this buffer. This must be
  681. * the same stream you called hadoopReadZero on.
  682. * @param buffer The buffer to release.
  683. */
  684. void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer);
  685. #ifdef __cplusplus
  686. }
  687. #endif
  688. #endif /*LIBHDFS_HDFS_H*/
  689. /**
  690. * vim: ts=4: sw=4: et
  691. */