fs.h 7.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201
  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 HADOOP_NATIVE_CORE_FS_H
  19. #define HADOOP_NATIVE_CORE_FS_H
  20. #include "fs/hdfs.h"
  21. #include <inttypes.h>
  22. struct hadoop_err;
  23. struct hadoop_uri;
  24. struct hconf;
  25. /**
  26. * fs.h
  27. *
  28. * This is the __private__ API for native Hadoop filesystems. (The public API
  29. * is in hdfs.h.) Native Hadoop filesystems such as JniFS or NDFS implement the
  30. * APIs in this file to provide a uniform experience to users.
  31. *
  32. * The general pattern here is:
  33. * 1. The client makes a call to libhdfs API
  34. * 2. fs.c locates the appropriate function in hadoop_fs_ops and calls it.
  35. * 3. Some filesystem-specific code implements the operation.
  36. *
  37. * In C, it is always safe to typecast a structure to the type of the first
  38. * element. This allows fs.c to treat hdfsFS instances as if they were
  39. * instances of struct hadoop_file_base. Other structures with "base" in the
  40. * name are intended to be used similarly. This functionality is similar in
  41. * many ways to how "base classes" operate in Java. The derived class contains
  42. * all the elements of the base class, plus some more.
  43. *
  44. * The structure definitions in this file are private, and users of this library
  45. * will not be able to access them. This file will not be packaged or
  46. * distributed... only hdfs.h will. Thus, it is safe to change any of the APIs
  47. * or types in this file without creating compatibility problems.
  48. */
  49. /**
  50. * Hadoop filesystem types.
  51. */
  52. enum hadoop_fs_ty {
  53. HADOOP_FS_TY_JNI = 0,
  54. HADOOP_FS_TY_NDFS = 1,
  55. HADOOP_FS_TY_NUM,
  56. };
  57. /**
  58. * Base data for Hadoop files.
  59. */
  60. struct hadoop_file_base {
  61. // The type of filesystem this file was created by.
  62. enum hadoop_fs_ty ty;
  63. };
  64. /**
  65. * Base data for Hadoop FileSystem objects.
  66. */
  67. struct hadoop_fs_base {
  68. // The type of this filesystem.
  69. enum hadoop_fs_ty ty;
  70. };
  71. /**
  72. * Base data for Hadoop Zero-Copy Read objects.
  73. */
  74. struct hadoopRzOptions {
  75. // The name of the ByteBufferPool class we should use when doing a zero-copy
  76. // read.
  77. char *pool_name;
  78. // Non-zero to always skip checksums.
  79. int skip_checksums;
  80. // If non-null, this callback will be invoked to tear down the cached data
  81. // inside this options structure during hadoopRzOptionsFree.
  82. void (*cache_teardown_cb)(void *);
  83. // The cached data inside this options structure.
  84. void *cache;
  85. };
  86. /**
  87. * Base data for Hadoop Zero-Copy Read buffers.
  88. */
  89. struct hadoop_rz_buffer_base {
  90. // The base address the client can start reading at.
  91. void *ptr;
  92. // The maximum valid length of this buffer.
  93. int32_t length;
  94. };
  95. struct hdfsBuilderConfOpt {
  96. struct hdfsBuilderConfOpt *next;
  97. const char *key;
  98. const char *val;
  99. };
  100. /**
  101. * A builder used to create Hadoop filesystem instances.
  102. */
  103. struct hdfsBuilder {
  104. const char *nn;
  105. uint16_t port;
  106. const char *kerbTicketCachePath;
  107. const char *userName;
  108. struct hdfsBuilderConfOpt *opts;
  109. struct hconf *hconf;
  110. struct hadoop_uri *uri;
  111. };
  112. /**
  113. * Operations which a libhadoopfs filesystem must implement.
  114. */
  115. struct hadoop_fs_ops {
  116. const char * const name;
  117. int (*file_is_open_for_read)(struct hdfsFile_internal *file);
  118. int (*file_is_open_for_write)(struct hdfsFile_internal * file);
  119. int (*get_read_statistics)(struct hdfsFile_internal *file,
  120. struct hdfsReadStatistics **stats);
  121. struct hadoop_err *(*connect)(struct hdfsBuilder *bld,
  122. struct hdfs_internal **fs);
  123. int (*disconnect)(struct hdfs_internal *fs);
  124. struct hdfsFile_internal *(*open)(struct hdfs_internal *fs,
  125. const char* uri, int flags, int bufferSize, short replication,
  126. int32_t blocksize);
  127. int (*close)(struct hdfs_internal *fs, struct hdfsFile_internal *file);
  128. int (*exists)(struct hdfs_internal *fs, const char *uri);
  129. int (*seek)(struct hdfs_internal *fs, struct hdfsFile_internal *file,
  130. int64_t desiredPos);
  131. int64_t (*tell)(struct hdfs_internal *fs, struct hdfsFile_internal *file);
  132. int32_t (*read)(struct hdfs_internal *fs, struct hdfsFile_internal *file,
  133. void* buffer, int32_t length);
  134. int32_t (*pread)(struct hdfs_internal *fs, struct hdfsFile_internal *file,
  135. int64_t position, void *buffer, int32_t length);
  136. int32_t (*write)(struct hdfs_internal *fs, struct hdfsFile_internal *file,
  137. const void* buffer, int32_t length);
  138. int (*flush)(struct hdfs_internal *fs, struct hdfsFile_internal *file);
  139. int (*hflush)(struct hdfs_internal *fs, struct hdfsFile_internal *file);
  140. int (*hsync)(struct hdfs_internal *fs, struct hdfsFile_internal *file);
  141. int (*available)(struct hdfs_internal * fs, struct hdfsFile_internal *file);
  142. int (*copy)(struct hdfs_internal *srcFS, const char *src,
  143. struct hdfs_internal *dstFS, const char *dst);
  144. int (*move)(struct hdfs_internal *srcFS, const char *src,
  145. struct hdfs_internal *dstFS, const char *dst);
  146. int (*unlink)(struct hdfs_internal *fs, const char *path, int recursive);
  147. int (*rename)(struct hdfs_internal *fs, const char *old_uri,
  148. const char* new_uri);
  149. char* (*get_working_directory)(struct hdfs_internal *fs, char *buffer,
  150. size_t bufferSize);
  151. int (*set_working_directory)(struct hdfs_internal *fs, const char* uri);
  152. int (*mkdir)(struct hdfs_internal *fs, const char* uri);
  153. int (*set_replication)(struct hdfs_internal *fs, const char* uri,
  154. int16_t replication);
  155. hdfsFileInfo *(*list_directory)(struct hdfs_internal *fs,
  156. const char* uri, int *numEntries);
  157. hdfsFileInfo *(*get_path_info)(struct hdfs_internal *fs, const char* uri);
  158. hdfsFileInfo *(*stat)(struct hdfs_internal *fs, const char* uri);
  159. void (*free_file_info)(hdfsFileInfo *, int numEntries);
  160. char*** (*get_hosts)(struct hdfs_internal *fs, const char* uri,
  161. int64_t start, int64_t length);
  162. int64_t (*get_default_block_size)(struct hdfs_internal *fs);
  163. int64_t (*get_default_block_size_at_path)(struct hdfs_internal *fs,
  164. const char *uri);
  165. int64_t (*get_capacity)(struct hdfs_internal *fs);
  166. int64_t (*get_used)(struct hdfs_internal *fs);
  167. int (*chown)(struct hdfs_internal *fs, const char *uri, const char *owner,
  168. const char *group);
  169. int (*chmod)(struct hdfs_internal *fs, const char* uri, short mode);
  170. int (*utime)(struct hdfs_internal *fs, const char* uri,
  171. int64_t mtime, int64_t atime);
  172. struct hadoopRzBuffer* (*read_zero)(struct hdfsFile_internal *file,
  173. struct hadoopRzOptions *opts, int32_t maxLength);
  174. void (*rz_buffer_free)(struct hdfsFile_internal *file,
  175. struct hadoopRzBuffer *buffer);
  176. // For testing
  177. int (*file_uses_direct_read)(struct hdfsFile_internal *fs);
  178. void (*file_disable_direct_read)(struct hdfsFile_internal *file);
  179. };
  180. #endif
  181. // vim: ts=4:sw=4:et