hdfs.h 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485
  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 <sys/types.h>
  21. #include <sys/stat.h>
  22. #include <fcntl.h>
  23. #include <stdio.h>
  24. #include <stdint.h>
  25. #include <string.h>
  26. #include <stdlib.h>
  27. #include <time.h>
  28. #include <errno.h>
  29. #include <jni.h>
  30. #ifndef O_RDONLY
  31. #define O_RDONLY 1
  32. #endif
  33. #ifndef O_WRONLY
  34. #define O_WRONLY 2
  35. #endif
  36. #ifndef EINTERNAL
  37. #define EINTERNAL 255
  38. #endif
  39. /** All APIs set errno to meaningful values */
  40. #ifdef __cplusplus
  41. extern "C" {
  42. #endif
  43. /**
  44. * Some utility decls used in libhdfs.
  45. */
  46. typedef int32_t tSize; /// size of data for read/write io ops
  47. typedef time_t tTime; /// time type in seconds
  48. typedef int64_t tOffset;/// offset within the file
  49. typedef uint16_t tPort; /// port
  50. typedef enum tObjectKind {
  51. kObjectKindFile = 'F',
  52. kObjectKindDirectory = 'D',
  53. } tObjectKind;
  54. /**
  55. * The C reflection of org.apache.org.hadoop.FileSystem .
  56. */
  57. typedef void* hdfsFS;
  58. /**
  59. * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
  60. */
  61. enum hdfsStreamType
  62. {
  63. UNINITIALIZED = 0,
  64. INPUT = 1,
  65. OUTPUT = 2,
  66. };
  67. // Bit fields for hdfsFile_internal flags
  68. #define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
  69. /**
  70. * The 'file-handle' to a file in hdfs.
  71. */
  72. struct hdfsFile_internal {
  73. void* file;
  74. enum hdfsStreamType type;
  75. uint32_t flags;
  76. };
  77. typedef struct hdfsFile_internal* hdfsFile;
  78. /**
  79. * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
  80. * Connect to the hdfs.
  81. * @param host A string containing either a host name, or an ip address
  82. * of the namenode of a hdfs cluster. 'host' should be passed as NULL if
  83. * you want to connect to local filesystem. 'host' should be passed as
  84. * 'default' (and port as 0) to used the 'configured' filesystem
  85. * (core-site/core-default.xml).
  86. * @param port The port on which the server is listening.
  87. * @param user the user name (this is hadoop domain user). Or NULL is equivelant to hhdfsConnect(host, port)
  88. * @return Returns a handle to the filesystem or NULL on error.
  89. */
  90. hdfsFS hdfsConnectAsUser(const char* host, tPort port, const char *user);
  91. /**
  92. * hdfsConnect - Connect to a hdfs file system.
  93. * Connect to the hdfs.
  94. * @param host A string containing either a host name, or an ip address
  95. * of the namenode of a hdfs cluster. 'host' should be passed as NULL if
  96. * you want to connect to local filesystem. 'host' should be passed as
  97. * 'default' (and port as 0) to used the 'configured' filesystem
  98. * (core-site/core-default.xml).
  99. * @param port The port on which the server is listening.
  100. * @return Returns a handle to the filesystem or NULL on error.
  101. */
  102. hdfsFS hdfsConnect(const char* host, tPort port);
  103. /**
  104. * This are the same as hdfsConnectAsUser except that every invocation returns a new FileSystem handle.
  105. * Applications should call a hdfsDisconnect for every call to hdfsConnectAsUserNewInstance.
  106. */
  107. hdfsFS hdfsConnectAsUserNewInstance(const char* host, tPort port, const char *user );
  108. hdfsFS hdfsConnectNewInstance(const char* host, tPort port);
  109. hdfsFS hdfsConnectPath(const char* uri);
  110. /**
  111. * hdfsDisconnect - Disconnect from the hdfs file system.
  112. * Disconnect from hdfs.
  113. * @param fs The configured filesystem handle.
  114. * @return Returns 0 on success, -1 on error.
  115. */
  116. int hdfsDisconnect(hdfsFS fs);
  117. /**
  118. * hdfsOpenFile - Open a hdfs file in given mode.
  119. * @param fs The configured filesystem handle.
  120. * @param path The full path to the file.
  121. * @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),
  122. * 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.
  123. * @param bufferSize Size of buffer for read/write - pass 0 if you want
  124. * to use the default configured values.
  125. * @param replication Block replication - pass 0 if you want to use
  126. * the default configured values.
  127. * @param blocksize Size of block - pass 0 if you want to use the
  128. * default configured values.
  129. * @return Returns the handle to the open file or NULL on error.
  130. */
  131. hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
  132. int bufferSize, short replication, tSize blocksize);
  133. /**
  134. * hdfsCloseFile - Close an open file.
  135. * @param fs The configured filesystem handle.
  136. * @param file The file handle.
  137. * @return Returns 0 on success, -1 on error.
  138. */
  139. int hdfsCloseFile(hdfsFS fs, hdfsFile file);
  140. /**
  141. * hdfsExists - Checks if a given path exsits on the filesystem
  142. * @param fs The configured filesystem handle.
  143. * @param path The path to look for
  144. * @return Returns 0 on success, -1 on error.
  145. */
  146. int hdfsExists(hdfsFS fs, const char *path);
  147. /**
  148. * hdfsSeek - Seek to given offset in file.
  149. * This works only for files opened in read-only mode.
  150. * @param fs The configured filesystem handle.
  151. * @param file The file handle.
  152. * @param desiredPos Offset into the file to seek into.
  153. * @return Returns 0 on success, -1 on error.
  154. */
  155. int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos);
  156. /**
  157. * hdfsTell - Get the current offset in the file, in bytes.
  158. * @param fs The configured filesystem handle.
  159. * @param file The file handle.
  160. * @return Current offset, -1 on error.
  161. */
  162. tOffset hdfsTell(hdfsFS fs, hdfsFile file);
  163. /**
  164. * hdfsRead - Read data from an open file.
  165. * @param fs The configured filesystem handle.
  166. * @param file The file handle.
  167. * @param buffer The buffer to copy read bytes into.
  168. * @param length The length of the buffer.
  169. * @return Returns the number of bytes actually read, possibly less
  170. * than than length;-1 on error.
  171. */
  172. tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length);
  173. /**
  174. * hdfsPread - Positional read of data from an open file.
  175. * @param fs The configured filesystem handle.
  176. * @param file The file handle.
  177. * @param position Position from which to read
  178. * @param buffer The buffer to copy read bytes into.
  179. * @param length The length of the buffer.
  180. * @return Returns the number of bytes actually read, possibly less than
  181. * than length;-1 on error.
  182. */
  183. tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
  184. void* buffer, tSize length);
  185. /**
  186. * hdfsWrite - Write data into an open file.
  187. * @param fs The configured filesystem handle.
  188. * @param file The file handle.
  189. * @param buffer The data.
  190. * @param length The no. of bytes to write.
  191. * @return Returns the number of bytes written, -1 on error.
  192. */
  193. tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
  194. tSize length);
  195. /**
  196. * hdfsWrite - Flush the data.
  197. * @param fs The configured filesystem handle.
  198. * @param file The file handle.
  199. * @return Returns 0 on success, -1 on error.
  200. */
  201. int hdfsFlush(hdfsFS fs, hdfsFile file);
  202. /**
  203. * hdfsHFlush - Flush out the data in client's user buffer. After the
  204. * return of this call, new readers will see the data.
  205. * @param fs configured filesystem handle
  206. * @param file file handle
  207. * @return 0 on success, -1 on error and sets errno
  208. */
  209. int hdfsHFlush(hdfsFS fs, hdfsFile file);
  210. /**
  211. * hdfsAvailable - Number of bytes that can be read from this
  212. * input stream without blocking.
  213. * @param fs The configured filesystem handle.
  214. * @param file The file handle.
  215. * @return Returns available bytes; -1 on error.
  216. */
  217. int hdfsAvailable(hdfsFS fs, hdfsFile file);
  218. /**
  219. * hdfsCopy - Copy file from one filesystem to another.
  220. * @param srcFS The handle to source filesystem.
  221. * @param src The path of source file.
  222. * @param dstFS The handle to destination filesystem.
  223. * @param dst The path of destination file.
  224. * @return Returns 0 on success, -1 on error.
  225. */
  226. int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  227. /**
  228. * hdfsMove - Move file from one filesystem to another.
  229. * @param srcFS The handle to source filesystem.
  230. * @param src The path of source file.
  231. * @param dstFS The handle to destination filesystem.
  232. * @param dst The path of destination file.
  233. * @return Returns 0 on success, -1 on error.
  234. */
  235. int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  236. /**
  237. * hdfsDelete - Delete file.
  238. * @param fs The configured filesystem handle.
  239. * @param path The path of the file.
  240. * @param recursive if path is a directory and set to
  241. * non-zero, the directory is deleted else throws an exception. In
  242. * case of a file the recursive argument is irrelevant.
  243. * @return Returns 0 on success, -1 on error.
  244. */
  245. int hdfsDelete(hdfsFS fs, const char* path, int recursive);
  246. /**
  247. * hdfsRename - Rename file.
  248. * @param fs The configured filesystem handle.
  249. * @param oldPath The path of the source file.
  250. * @param newPath The path of the destination file.
  251. * @return Returns 0 on success, -1 on error.
  252. */
  253. int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
  254. /**
  255. * hdfsGetWorkingDirectory - Get the current working directory for
  256. * the given filesystem.
  257. * @param fs The configured filesystem handle.
  258. * @param buffer The user-buffer to copy path of cwd into.
  259. * @param bufferSize The length of user-buffer.
  260. * @return Returns buffer, NULL on error.
  261. */
  262. char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
  263. /**
  264. * hdfsSetWorkingDirectory - Set the working directory. All relative
  265. * paths will be resolved relative to it.
  266. * @param fs The configured filesystem handle.
  267. * @param path The path of the new 'cwd'.
  268. * @return Returns 0 on success, -1 on error.
  269. */
  270. int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
  271. /**
  272. * hdfsCreateDirectory - Make the given file and all non-existent
  273. * parents into directories.
  274. * @param fs The configured filesystem handle.
  275. * @param path The path of the directory.
  276. * @return Returns 0 on success, -1 on error.
  277. */
  278. int hdfsCreateDirectory(hdfsFS fs, const char* path);
  279. /**
  280. * hdfsSetReplication - Set the replication of the specified
  281. * file to the supplied value
  282. * @param fs The configured filesystem handle.
  283. * @param path The path of the file.
  284. * @return Returns 0 on success, -1 on error.
  285. */
  286. int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication);
  287. /**
  288. * hdfsFileInfo - Information about a file/directory.
  289. */
  290. typedef struct {
  291. tObjectKind mKind; /* file or directory */
  292. char *mName; /* the name of the file */
  293. tTime mLastMod; /* the last modification time for the file in seconds */
  294. tOffset mSize; /* the size of the file in bytes */
  295. short mReplication; /* the count of replicas */
  296. tOffset mBlockSize; /* the block size for the file */
  297. char *mOwner; /* the owner of the file */
  298. char *mGroup; /* the group associated with the file */
  299. short mPermissions; /* the permissions associated with the file */
  300. tTime mLastAccess; /* the last access time for the file in seconds */
  301. } hdfsFileInfo;
  302. /**
  303. * hdfsListDirectory - Get list of files/directories for a given
  304. * directory-path. hdfsFreeFileInfo should be called to deallocate memory.
  305. * @param fs The configured filesystem handle.
  306. * @param path The path of the directory.
  307. * @param numEntries Set to the number of files/directories in path.
  308. * @return Returns a dynamically-allocated array of hdfsFileInfo
  309. * objects; NULL on error.
  310. */
  311. hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
  312. int *numEntries);
  313. /**
  314. * hdfsGetPathInfo - Get information about a path as a (dynamically
  315. * allocated) single hdfsFileInfo struct. hdfsFreeFileInfo should be
  316. * called when the pointer is no longer needed.
  317. * @param fs The configured filesystem handle.
  318. * @param path The path of the file.
  319. * @return Returns a dynamically-allocated hdfsFileInfo object;
  320. * NULL on error.
  321. */
  322. hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
  323. /**
  324. * hdfsFreeFileInfo - Free up the hdfsFileInfo array (including fields)
  325. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  326. * objects.
  327. * @param numEntries The size of the array.
  328. */
  329. void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
  330. /**
  331. * hdfsGetHosts - Get hostnames where a particular block (determined by
  332. * pos & blocksize) of a file is stored. The last element in the array
  333. * is NULL. Due to replication, a single block could be present on
  334. * multiple hosts.
  335. * @param fs The configured filesystem handle.
  336. * @param path The path of the file.
  337. * @param start The start of the block.
  338. * @param length The length of the block.
  339. * @return Returns a dynamically-allocated 2-d array of blocks-hosts;
  340. * NULL on error.
  341. */
  342. char*** hdfsGetHosts(hdfsFS fs, const char* path,
  343. tOffset start, tOffset length);
  344. /**
  345. * hdfsFreeHosts - Free up the structure returned by hdfsGetHosts
  346. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  347. * objects.
  348. * @param numEntries The size of the array.
  349. */
  350. void hdfsFreeHosts(char ***blockHosts);
  351. /**
  352. * hdfsGetDefaultBlockSize - Get the optimum blocksize.
  353. * @param fs The configured filesystem handle.
  354. * @return Returns the blocksize; -1 on error.
  355. */
  356. tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
  357. /**
  358. * hdfsGetCapacity - Return the raw capacity of the filesystem.
  359. * @param fs The configured filesystem handle.
  360. * @return Returns the raw-capacity; -1 on error.
  361. */
  362. tOffset hdfsGetCapacity(hdfsFS fs);
  363. /**
  364. * hdfsGetUsed - Return the total raw size of all files in the filesystem.
  365. * @param fs The configured filesystem handle.
  366. * @return Returns the total-size; -1 on error.
  367. */
  368. tOffset hdfsGetUsed(hdfsFS fs);
  369. /**
  370. * hdfsChown
  371. * @param fs The configured filesystem handle.
  372. * @param path the path to the file or directory
  373. * @param owner this is a string in Hadoop land. Set to null or "" if only setting group
  374. * @param group this is a string in Hadoop land. Set to null or "" if only setting user
  375. * @return 0 on success else -1
  376. */
  377. int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group);
  378. /**
  379. * hdfsChmod
  380. * @param fs The configured filesystem handle.
  381. * @param path the path to the file or directory
  382. * @param mode the bitmask to set it to
  383. * @return 0 on success else -1
  384. */
  385. int hdfsChmod(hdfsFS fs, const char* path, short mode);
  386. /**
  387. * hdfsUtime
  388. * @param fs The configured filesystem handle.
  389. * @param path the path to the file or directory
  390. * @param mtime new modification time or 0 for only set access time in seconds
  391. * @param atime new access time or 0 for only set modification time in seconds
  392. * @return 0 on success else -1
  393. */
  394. int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
  395. #ifdef __cplusplus
  396. }
  397. #endif
  398. #endif /*LIBHDFS_HDFS_H*/
  399. /**
  400. * vim: ts=4: sw=4: et
  401. */