hdfs.h 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422
  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
  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. /**
  68. * The 'file-handle' to a file in hdfs.
  69. */
  70. struct hdfsFile_internal {
  71. void* file;
  72. enum hdfsStreamType type;
  73. };
  74. typedef struct hdfsFile_internal* hdfsFile;
  75. /**
  76. * hdfsConnect - Connect to a hdfs file system.
  77. * Connect to the hdfs.
  78. * @param host A string containing either a host name, or an ip address
  79. * of the namenode of a hdfs cluster. 'host' should be passed as NULL if
  80. * you want to connect to local filesystem. 'host' should be passed as
  81. * 'default' (and port as 0) to used the 'configured' filesystem
  82. * (hadoop-site/hadoop-default.xml).
  83. * @param port The port on which the server is listening.
  84. * @return Returns a handle to the filesystem or NULL on error.
  85. */
  86. hdfsFS hdfsConnect(const char* host, tPort port);
  87. /**
  88. * hdfsDisconnect - Disconnect from the hdfs file system.
  89. * Disconnect from hdfs.
  90. * @param fs The configured filesystem handle.
  91. * @return Returns 0 on success, -1 on error.
  92. */
  93. int hdfsDisconnect(hdfsFS fs);
  94. /**
  95. * hdfsOpenFile - Open a hdfs file in given mode.
  96. * @param fs The configured filesystem handle.
  97. * @param path The full path to the file.
  98. * @param flags Either O_RDONLY or O_WRONLY, for read-only or write-only.
  99. * @param bufferSize Size of buffer for read/write - pass 0 if you want
  100. * to use the default configured values.
  101. * @param replication Block replication - pass 0 if you want to use
  102. * the default configured values.
  103. * @param blocksize Size of block - pass 0 if you want to use the
  104. * default configured values.
  105. * @return Returns the handle to the open file or NULL on error.
  106. */
  107. hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
  108. int bufferSize, short replication, tSize blocksize);
  109. /**
  110. * hdfsCloseFile - Close an open file.
  111. * @param fs The configured filesystem handle.
  112. * @param file The file handle.
  113. * @return Returns 0 on success, -1 on error.
  114. */
  115. int hdfsCloseFile(hdfsFS fs, hdfsFile file);
  116. /**
  117. * hdfsExists - Checks if a given path exsits on the filesystem
  118. * @param fs The configured filesystem handle.
  119. * @param path The path to look for
  120. * @return Returns 0 on success, -1 on error.
  121. */
  122. int hdfsExists(hdfsFS fs, const char *path);
  123. /**
  124. * hdfsSeek - Seek to given offset in file.
  125. * This works only for files opened in read-only mode.
  126. * @param fs The configured filesystem handle.
  127. * @param file The file handle.
  128. * @param desiredPos Offset into the file to seek into.
  129. * @return Returns 0 on success, -1 on error.
  130. */
  131. int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos);
  132. /**
  133. * hdfsTell - Get the current offset in the file, in bytes.
  134. * @param fs The configured filesystem handle.
  135. * @param file The file handle.
  136. * @return Current offset, -1 on error.
  137. */
  138. tOffset hdfsTell(hdfsFS fs, hdfsFile file);
  139. /**
  140. * hdfsRead - Read data from an open file.
  141. * @param fs The configured filesystem handle.
  142. * @param file The file handle.
  143. * @param buffer The buffer to copy read bytes into.
  144. * @param length The length of the buffer.
  145. * @return Returns the number of bytes actually read, possibly less
  146. * than than length;-1 on error.
  147. */
  148. tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length);
  149. /**
  150. * hdfsPread - Positional read of data from an open file.
  151. * @param fs The configured filesystem handle.
  152. * @param file The file handle.
  153. * @param position Position from which to read
  154. * @param buffer The buffer to copy read bytes into.
  155. * @param length The length of the buffer.
  156. * @return Returns the number of bytes actually read, possibly less than
  157. * than length;-1 on error.
  158. */
  159. tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
  160. void* buffer, tSize length);
  161. /**
  162. * hdfsWrite - Write data into an open file.
  163. * @param fs The configured filesystem handle.
  164. * @param file The file handle.
  165. * @param buffer The data.
  166. * @param length The no. of bytes to write.
  167. * @return Returns the number of bytes written, -1 on error.
  168. */
  169. tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
  170. tSize length);
  171. /**
  172. * hdfsWrite - Flush the data.
  173. * @param fs The configured filesystem handle.
  174. * @param file The file handle.
  175. * @return Returns 0 on success, -1 on error.
  176. */
  177. int hdfsFlush(hdfsFS fs, hdfsFile file);
  178. /**
  179. * hdfsAvailable - Number of bytes that can be read from this
  180. * input stream without blocking.
  181. * @param fs The configured filesystem handle.
  182. * @param file The file handle.
  183. * @return Returns available bytes; -1 on error.
  184. */
  185. int hdfsAvailable(hdfsFS fs, hdfsFile file);
  186. /**
  187. * hdfsCopy - Copy file from one filesystem to another.
  188. * @param srcFS The handle to source filesystem.
  189. * @param src The path of source file.
  190. * @param dstFS The handle to destination filesystem.
  191. * @param dst The path of destination file.
  192. * @return Returns 0 on success, -1 on error.
  193. */
  194. int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  195. /**
  196. * hdfsMove - Move file from one filesystem to another.
  197. * @param srcFS The handle to source filesystem.
  198. * @param src The path of source file.
  199. * @param dstFS The handle to destination filesystem.
  200. * @param dst The path of destination file.
  201. * @return Returns 0 on success, -1 on error.
  202. */
  203. int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
  204. /**
  205. * hdfsDelete - Delete file.
  206. * @param fs The configured filesystem handle.
  207. * @param path The path of the file.
  208. * @return Returns 0 on success, -1 on error.
  209. */
  210. int hdfsDelete(hdfsFS fs, const char* path);
  211. /**
  212. * hdfsRename - Rename file.
  213. * @param fs The configured filesystem handle.
  214. * @param oldPath The path of the source file.
  215. * @param newPath The path of the destination file.
  216. * @return Returns 0 on success, -1 on error.
  217. */
  218. int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
  219. /**
  220. * hdfsLock - Obtain a lock on the file.
  221. * @param fs The configured filesystem handle.
  222. * @param path The path of the file.
  223. * @param shared Shared/exclusive lock-type.
  224. * @return Returns 0 on success, -1 on error.
  225. */
  226. int hdfsLock(hdfsFS fs, const char* path, int shared);
  227. /**
  228. * hdfsReleaseLock - Release the lock.
  229. * @param fs The configured filesystem handle.
  230. * @param path The path of the file.
  231. * @return Returns 0 on success, -1 on error.
  232. */
  233. int hdfsReleaseLock(hdfsFS fs, const char* path);
  234. /**
  235. * hdfsGetWorkingDirectory - Get the current working directory for
  236. * the given filesystem.
  237. * @param fs The configured filesystem handle.
  238. * @param buffer The user-buffer to copy path of cwd into.
  239. * @param bufferSize The length of user-buffer.
  240. * @return Returns buffer, NULL on error.
  241. */
  242. char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
  243. /**
  244. * hdfsSetWorkingDirectory - Set the working directory. All relative
  245. * paths will be resolved relative to it.
  246. * @param fs The configured filesystem handle.
  247. * @param path The path of the new 'cwd'.
  248. * @return Returns 0 on success, -1 on error.
  249. */
  250. int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
  251. /**
  252. * hdfsCreateDirectory - Make the given file and all non-existent
  253. * parents into directories.
  254. * @param fs The configured filesystem handle.
  255. * @param path The path of the directory.
  256. * @return Returns 0 on success, -1 on error.
  257. */
  258. int hdfsCreateDirectory(hdfsFS fs, const char* path);
  259. /**
  260. * hdfsFileInfo - Information about a file/directory.
  261. */
  262. typedef struct {
  263. tObjectKind mKind; /* file or directory */
  264. char *mName; /* the name of the file */
  265. tTime mCreationTime; /* the creation time for the file*/
  266. tOffset mSize; /* the size of the file in bytes */
  267. int replicaCount; /* the count of replicas */
  268. } hdfsFileInfo;
  269. /**
  270. * hdfsListDirectory - Get list of files/directories for a given
  271. * directory-path. freehdfsFileInfo should be called to deallocate memory.
  272. * @param fs The configured filesystem handle.
  273. * @param path The path of the directory.
  274. * @param numEntries Set to the number of files/directories in path.
  275. * @return Returns a dynamically-allocated array of hdfsFileInfo
  276. * objects; NULL on error.
  277. */
  278. hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
  279. int *numEntries);
  280. /**
  281. * hdfsGetPathInfo - Get information about a path as a (dynamically
  282. * allocated) single hdfsFileInfo struct. freehdfsFileInfo should be
  283. * called when the pointer is no longer needed.
  284. * @param fs The configured filesystem handle.
  285. * @param path The path of the file.
  286. * @return Returns a dynamically-allocated hdfsFileInfo object;
  287. * NULL on error.
  288. */
  289. hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
  290. /**
  291. * hdfsFreeFileInfo - Free up the hdfsFileInfo array (including fields)
  292. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  293. * objects.
  294. * @param numEntries The size of the array.
  295. */
  296. void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
  297. /**
  298. * hdfsGetHosts - Get hostnames where a particular block (determined by
  299. * pos & blocksize) of a file is stored. The last element in the array
  300. * is NULL. Due to replication, a single block could be present on
  301. * multiple hosts.
  302. * @param fs The configured filesystem handle.
  303. * @param path The path of the file.
  304. * @param start The start of the block.
  305. * @param length The length of the block.
  306. * @return Returns a dynamically-allocated 2-d array of blocks-hosts;
  307. * NULL on error.
  308. */
  309. char*** hdfsGetHosts(hdfsFS fs, const char* path,
  310. tOffset start, tOffset length);
  311. /**
  312. * hdfsFreeHosts - Free up the structure returned by hdfsGetHosts
  313. * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
  314. * objects.
  315. * @param numEntries The size of the array.
  316. */
  317. void hdfsFreeHosts(char ***blockHosts);
  318. /**
  319. * hdfsGetDefaultBlockSize - Get the optimum blocksize.
  320. * @param fs The configured filesystem handle.
  321. * @return Returns the blocksize; -1 on error.
  322. */
  323. tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
  324. /**
  325. * hdfsGetCapacity - Return the raw capacity of the filesystem.
  326. * @param fs The configured filesystem handle.
  327. * @return Returns the raw-capacity; -1 on error.
  328. */
  329. tOffset hdfsGetCapacity(hdfsFS fs);
  330. /**
  331. * hdfsGetUsed - Return the total raw size of all files in the filesystem.
  332. * @param fs The configured filesystem handle.
  333. * @return Returns the total-size; -1 on error.
  334. */
  335. tOffset hdfsGetUsed(hdfsFS fs);
  336. #ifdef __cplusplus
  337. }
  338. #endif
  339. #endif /*LIBHDFS_HDFS_H*/
  340. /**
  341. * vim: ts=4: sw=4: et
  342. */