123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799 |
- /**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- #ifndef LIBHDFS_HDFS_H
- #define LIBHDFS_HDFS_H
- #include <errno.h> /* for EINTERNAL, etc. */
- #include <fcntl.h> /* for O_RDONLY, O_WRONLY */
- #include <stdint.h> /* for uint64_t, etc. */
- #include <time.h> /* for time_t */
- #ifndef O_RDONLY
- #define O_RDONLY 1
- #endif
- #ifndef O_WRONLY
- #define O_WRONLY 2
- #endif
- #ifndef EINTERNAL
- #define EINTERNAL 255
- #endif
- #define ELASTIC_BYTE_BUFFER_POOL_CLASS \
- "org/apache/hadoop/io/ElasticByteBufferPool"
- /** All APIs set errno to meaningful values */
- #ifdef __cplusplus
- extern "C" {
- #endif
- /**
- * Some utility decls used in libhdfs.
- */
- struct hdfsBuilder;
- typedef int32_t tSize; /// size of data for read/write io ops
- typedef time_t tTime; /// time type in seconds
- typedef int64_t tOffset;/// offset within the file
- typedef uint16_t tPort; /// port
- typedef enum tObjectKind {
- kObjectKindFile = 'F',
- kObjectKindDirectory = 'D',
- } tObjectKind;
- /**
- * The C reflection of org.apache.org.hadoop.FileSystem .
- */
- struct hdfs_internal;
- typedef struct hdfs_internal* hdfsFS;
-
- struct hdfsFile_internal;
- typedef struct hdfsFile_internal* hdfsFile;
- struct hadoopRzOptions;
- struct hadoopRzBuffer;
- /**
- * Determine if a file is open for read.
- *
- * @param file The HDFS file
- * @return 1 if the file is open for read; 0 otherwise
- */
- int hdfsFileIsOpenForRead(hdfsFile file);
- /**
- * Determine if a file is open for write.
- *
- * @param file The HDFS file
- * @return 1 if the file is open for write; 0 otherwise
- */
- int hdfsFileIsOpenForWrite(hdfsFile file);
- struct hdfsReadStatistics {
- uint64_t totalBytesRead;
- uint64_t totalLocalBytesRead;
- uint64_t totalShortCircuitBytesRead;
- uint64_t totalZeroCopyBytesRead;
- };
- /**
- * Get read statistics about a file. This is only applicable to files
- * opened for reading.
- *
- * @param file The HDFS file
- * @param stats (out parameter) on a successful return, the read
- * statistics. Unchanged otherwise. You must free the
- * returned statistics with hdfsFileFreeReadStatistics.
- * @return 0 if the statistics were successfully returned,
- * -1 otherwise. On a failure, please check errno against
- * ENOTSUP. webhdfs, LocalFilesystem, and so forth may
- * not support read statistics.
- */
- int hdfsFileGetReadStatistics(hdfsFile file,
- struct hdfsReadStatistics **stats);
- /**
- * @param stats HDFS read statistics for a file.
- *
- * @return the number of remote bytes read.
- */
- int64_t hdfsReadStatisticsGetRemoteBytesRead(
- const struct hdfsReadStatistics *stats);
- /**
- * Free some HDFS read statistics.
- *
- * @param stats The HDFS read statistics to free.
- */
- void hdfsFileFreeReadStatistics(struct hdfsReadStatistics *stats);
- /**
- * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
- * Connect to the hdfs.
- * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
- * @param port The port on which the server is listening.
- * @param user the user name (this is hadoop domain user). Or NULL is equivelant to hhdfsConnect(host, port)
- * @return Returns a handle to the filesystem or NULL on error.
- * @deprecated Use hdfsBuilderConnect instead.
- */
- hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user);
- /**
- * hdfsConnect - Connect to a hdfs file system.
- * Connect to the hdfs.
- * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
- * @param port The port on which the server is listening.
- * @return Returns a handle to the filesystem or NULL on error.
- * @deprecated Use hdfsBuilderConnect instead.
- */
- hdfsFS hdfsConnect(const char* nn, tPort port);
- /**
- * hdfsConnect - Connect to an hdfs file system.
- *
- * Forces a new instance to be created
- *
- * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
- * @param port The port on which the server is listening.
- * @param user The user name to use when connecting
- * @return Returns a handle to the filesystem or NULL on error.
- * @deprecated Use hdfsBuilderConnect instead.
- */
- hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user );
- /**
- * hdfsConnect - Connect to an hdfs file system.
- *
- * Forces a new instance to be created
- *
- * @param nn The NameNode. See hdfsBuilderSetNameNode for details.
- * @param port The port on which the server is listening.
- * @return Returns a handle to the filesystem or NULL on error.
- * @deprecated Use hdfsBuilderConnect instead.
- */
- hdfsFS hdfsConnectNewInstance(const char* nn, tPort port);
- /**
- * Connect to HDFS using the parameters defined by the builder.
- *
- * The HDFS builder will be freed, whether or not the connection was
- * successful.
- *
- * Every successful call to hdfsBuilderConnect should be matched with a call
- * to hdfsDisconnect, when the hdfsFS is no longer needed.
- *
- * @param bld The HDFS builder
- * @return Returns a handle to the filesystem, or NULL on error.
- */
- hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld);
- /**
- * Create an HDFS builder.
- *
- * @return The HDFS builder, or NULL on error.
- */
- struct hdfsBuilder *hdfsNewBuilder(void);
- /**
- * Force the builder to always create a new instance of the FileSystem,
- * rather than possibly finding one in the cache.
- *
- * @param bld The HDFS builder
- */
- void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld);
- /**
- * Set the HDFS NameNode to connect to.
- *
- * @param bld The HDFS builder
- * @param nn The NameNode to use.
- *
- * If the string given is 'default', the default NameNode
- * configuration will be used (from the XML configuration files)
- *
- * If NULL is given, a LocalFileSystem will be created.
- *
- * If the string starts with a protocol type such as file:// or
- * hdfs://, this protocol type will be used. If not, the
- * hdfs:// protocol type will be used.
- *
- * You may specify a NameNode port in the usual way by
- * passing a string of the format hdfs://<hostname>:<port>.
- * Alternately, you may set the port with
- * hdfsBuilderSetNameNodePort. However, you must not pass the
- * port in two different ways.
- */
- void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn);
- /**
- * Set the port of the HDFS NameNode to connect to.
- *
- * @param bld The HDFS builder
- * @param port The port.
- */
- void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port);
- /**
- * Set the username to use when connecting to the HDFS cluster.
- *
- * @param bld The HDFS builder
- * @param userName The user name. The string will be shallow-copied.
- */
- void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName);
- /**
- * Set the path to the Kerberos ticket cache to use when connecting to
- * the HDFS cluster.
- *
- * @param bld The HDFS builder
- * @param kerbTicketCachePath The Kerberos ticket cache path. The string
- * will be shallow-copied.
- */
- void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
- const char *kerbTicketCachePath);
- /**
- * Free an HDFS builder.
- *
- * It is normally not necessary to call this function since
- * hdfsBuilderConnect frees the builder.
- *
- * @param bld The HDFS builder
- */
- void hdfsFreeBuilder(struct hdfsBuilder *bld);
- /**
- * Set a configuration string for an HdfsBuilder.
- *
- * @param key The key to set.
- * @param val The value, or NULL to set no value.
- * This will be shallow-copied. You are responsible for
- * ensuring that it remains valid until the builder is
- * freed.
- *
- * @return 0 on success; nonzero error code otherwise.
- */
- int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
- const char *val);
- /**
- * Get a configuration string.
- *
- * @param key The key to find
- * @param val (out param) The value. This will be set to NULL if the
- * key isn't found. You must free this string with
- * hdfsConfStrFree.
- *
- * @return 0 on success; nonzero error code otherwise.
- * Failure to find the key is not an error.
- */
- int hdfsConfGetStr(const char *key, char **val);
- /**
- * Get a configuration integer.
- *
- * @param key The key to find
- * @param val (out param) The value. This will NOT be changed if the
- * key isn't found.
- *
- * @return 0 on success; nonzero error code otherwise.
- * Failure to find the key is not an error.
- */
- int hdfsConfGetInt(const char *key, int32_t *val);
- /**
- * Free a configuration string found with hdfsConfGetStr.
- *
- * @param val A configuration string obtained from hdfsConfGetStr
- */
- void hdfsConfStrFree(char *val);
- /**
- * hdfsDisconnect - Disconnect from the hdfs file system.
- * Disconnect from hdfs.
- * @param fs The configured filesystem handle.
- * @return Returns 0 on success, -1 on error.
- * Even if there is an error, the resources associated with the
- * hdfsFS will be freed.
- */
- int hdfsDisconnect(hdfsFS fs);
-
- /**
- * hdfsOpenFile - Open a hdfs file in given mode.
- * @param fs The configured filesystem handle.
- * @param path The full path to the file.
- * @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),
- * 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.
- * @param bufferSize Size of buffer for read/write - pass 0 if you want
- * to use the default configured values.
- * @param replication Block replication - pass 0 if you want to use
- * the default configured values.
- * @param blocksize Size of block - pass 0 if you want to use the
- * default configured values.
- * @return Returns the handle to the open file or NULL on error.
- */
- hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
- int bufferSize, short replication, tSize blocksize);
- /**
- * hdfsCloseFile - Close an open file.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @return Returns 0 on success, -1 on error.
- * On error, errno will be set appropriately.
- * If the hdfs file was valid, the memory associated with it will
- * be freed at the end of this call, even if there was an I/O
- * error.
- */
- int hdfsCloseFile(hdfsFS fs, hdfsFile file);
- /**
- * hdfsExists - Checks if a given path exsits on the filesystem
- * @param fs The configured filesystem handle.
- * @param path The path to look for
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsExists(hdfsFS fs, const char *path);
- /**
- * hdfsSeek - Seek to given offset in file.
- * This works only for files opened in read-only mode.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @param desiredPos Offset into the file to seek into.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos);
- /**
- * hdfsTell - Get the current offset in the file, in bytes.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @return Current offset, -1 on error.
- */
- tOffset hdfsTell(hdfsFS fs, hdfsFile file);
- /**
- * hdfsRead - Read data from an open file.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @param buffer The buffer to copy read bytes into.
- * @param length The length of the buffer.
- * @return On success, a positive number indicating how many bytes
- * were read.
- * On end-of-file, 0.
- * On error, -1. Errno will be set to the error code.
- * Just like the POSIX read function, hdfsRead will return -1
- * and set errno to EINTR if data is temporarily unavailable,
- * but we are not yet at the end of the file.
- */
- tSize hdfsRead(hdfsFS fs, hdfsFile file, void* buffer, tSize length);
- /**
- * hdfsPread - Positional read of data from an open file.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @param position Position from which to read
- * @param buffer The buffer to copy read bytes into.
- * @param length The length of the buffer.
- * @return See hdfsRead
- */
- tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position,
- void* buffer, tSize length);
- /**
- * hdfsWrite - Write data into an open file.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @param buffer The data.
- * @param length The no. of bytes to write.
- * @return Returns the number of bytes written, -1 on error.
- */
- tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer,
- tSize length);
- /**
- * hdfsWrite - Flush the data.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsFlush(hdfsFS fs, hdfsFile file);
- /**
- * hdfsHFlush - Flush out the data in client's user buffer. After the
- * return of this call, new readers will see the data.
- * @param fs configured filesystem handle
- * @param file file handle
- * @return 0 on success, -1 on error and sets errno
- */
- int hdfsHFlush(hdfsFS fs, hdfsFile file);
- /**
- * hdfsHSync - Similar to posix fsync, Flush out the data in client's
- * user buffer. all the way to the disk device (but the disk may have
- * it in its cache).
- * @param fs configured filesystem handle
- * @param file file handle
- * @return 0 on success, -1 on error and sets errno
- */
- int hdfsHSync(hdfsFS fs, hdfsFile file);
- /**
- * hdfsAvailable - Number of bytes that can be read from this
- * input stream without blocking.
- * @param fs The configured filesystem handle.
- * @param file The file handle.
- * @return Returns available bytes; -1 on error.
- */
- int hdfsAvailable(hdfsFS fs, hdfsFile file);
- /**
- * hdfsCopy - Copy file from one filesystem to another.
- * @param srcFS The handle to source filesystem.
- * @param src The path of source file.
- * @param dstFS The handle to destination filesystem.
- * @param dst The path of destination file.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsCopy(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
- /**
- * hdfsMove - Move file from one filesystem to another.
- * @param srcFS The handle to source filesystem.
- * @param src The path of source file.
- * @param dstFS The handle to destination filesystem.
- * @param dst The path of destination file.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst);
- /**
- * hdfsDelete - Delete file.
- * @param fs The configured filesystem handle.
- * @param path The path of the file.
- * @param recursive if path is a directory and set to
- * non-zero, the directory is deleted else throws an exception. In
- * case of a file the recursive argument is irrelevant.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsDelete(hdfsFS fs, const char* path, int recursive);
- /**
- * hdfsRename - Rename file.
- * @param fs The configured filesystem handle.
- * @param oldPath The path of the source file.
- * @param newPath The path of the destination file.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath);
- /**
- * hdfsGetWorkingDirectory - Get the current working directory for
- * the given filesystem.
- * @param fs The configured filesystem handle.
- * @param buffer The user-buffer to copy path of cwd into.
- * @param bufferSize The length of user-buffer.
- * @return Returns buffer, NULL on error.
- */
- char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize);
- /**
- * hdfsSetWorkingDirectory - Set the working directory. All relative
- * paths will be resolved relative to it.
- * @param fs The configured filesystem handle.
- * @param path The path of the new 'cwd'.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsSetWorkingDirectory(hdfsFS fs, const char* path);
- /**
- * hdfsCreateDirectory - Make the given file and all non-existent
- * parents into directories.
- * @param fs The configured filesystem handle.
- * @param path The path of the directory.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsCreateDirectory(hdfsFS fs, const char* path);
- /**
- * hdfsSetReplication - Set the replication of the specified
- * file to the supplied value
- * @param fs The configured filesystem handle.
- * @param path The path of the file.
- * @return Returns 0 on success, -1 on error.
- */
- int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication);
- /**
- * hdfsFileInfo - Information about a file/directory.
- */
- typedef struct {
- tObjectKind mKind; /* file or directory */
- char *mName; /* the name of the file */
- tTime mLastMod; /* the last modification time for the file in seconds */
- tOffset mSize; /* the size of the file in bytes */
- short mReplication; /* the count of replicas */
- tOffset mBlockSize; /* the block size for the file */
- char *mOwner; /* the owner of the file */
- char *mGroup; /* the group associated with the file */
- short mPermissions; /* the permissions associated with the file */
- tTime mLastAccess; /* the last access time for the file in seconds */
- } hdfsFileInfo;
- /**
- * hdfsListDirectory - Get list of files/directories for a given
- * directory-path. hdfsFreeFileInfo should be called to deallocate memory.
- * @param fs The configured filesystem handle.
- * @param path The path of the directory.
- * @param numEntries Set to the number of files/directories in path.
- * @return Returns a dynamically-allocated array of hdfsFileInfo
- * objects; NULL on error.
- */
- hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path,
- int *numEntries);
- /**
- * hdfsGetPathInfo - Get information about a path as a (dynamically
- * allocated) single hdfsFileInfo struct. hdfsFreeFileInfo should be
- * called when the pointer is no longer needed.
- * @param fs The configured filesystem handle.
- * @param path The path of the file.
- * @return Returns a dynamically-allocated hdfsFileInfo object;
- * NULL on error.
- */
- hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path);
- /**
- * hdfsFreeFileInfo - Free up the hdfsFileInfo array (including fields)
- * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
- * objects.
- * @param numEntries The size of the array.
- */
- void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries);
- /**
- * hdfsGetHosts - Get hostnames where a particular block (determined by
- * pos & blocksize) of a file is stored. The last element in the array
- * is NULL. Due to replication, a single block could be present on
- * multiple hosts.
- * @param fs The configured filesystem handle.
- * @param path The path of the file.
- * @param start The start of the block.
- * @param length The length of the block.
- * @return Returns a dynamically-allocated 2-d array of blocks-hosts;
- * NULL on error.
- */
- char*** hdfsGetHosts(hdfsFS fs, const char* path,
- tOffset start, tOffset length);
- /**
- * hdfsFreeHosts - Free up the structure returned by hdfsGetHosts
- * @param hdfsFileInfo The array of dynamically-allocated hdfsFileInfo
- * objects.
- * @param numEntries The size of the array.
- */
- void hdfsFreeHosts(char ***blockHosts);
- /**
- * hdfsGetDefaultBlockSize - Get the default blocksize.
- *
- * @param fs The configured filesystem handle.
- * @deprecated Use hdfsGetDefaultBlockSizeAtPath instead.
- *
- * @return Returns the default blocksize, or -1 on error.
- */
- tOffset hdfsGetDefaultBlockSize(hdfsFS fs);
- /**
- * hdfsGetDefaultBlockSizeAtPath - Get the default blocksize at the
- * filesystem indicated by a given path.
- *
- * @param fs The configured filesystem handle.
- * @param path The given path will be used to locate the actual
- * filesystem. The full path does not have to exist.
- *
- * @return Returns the default blocksize, or -1 on error.
- */
- tOffset hdfsGetDefaultBlockSizeAtPath(hdfsFS fs, const char *path);
- /**
- * hdfsGetCapacity - Return the raw capacity of the filesystem.
- * @param fs The configured filesystem handle.
- * @return Returns the raw-capacity; -1 on error.
- */
- tOffset hdfsGetCapacity(hdfsFS fs);
- /**
- * hdfsGetUsed - Return the total raw size of all files in the filesystem.
- * @param fs The configured filesystem handle.
- * @return Returns the total-size; -1 on error.
- */
- tOffset hdfsGetUsed(hdfsFS fs);
- /**
- * Change the user and/or group of a file or directory.
- *
- * @param fs The configured filesystem handle.
- * @param path the path to the file or directory
- * @param owner User string. Set to NULL for 'no change'
- * @param group Group string. Set to NULL for 'no change'
- * @return 0 on success else -1
- */
- int hdfsChown(hdfsFS fs, const char* path, const char *owner,
- const char *group);
- /**
- * hdfsChmod
- * @param fs The configured filesystem handle.
- * @param path the path to the file or directory
- * @param mode the bitmask to set it to
- * @return 0 on success else -1
- */
- int hdfsChmod(hdfsFS fs, const char* path, short mode);
- /**
- * hdfsUtime
- * @param fs The configured filesystem handle.
- * @param path the path to the file or directory
- * @param mtime new modification time or -1 for no change
- * @param atime new access time or -1 for no change
- * @return 0 on success else -1
- */
- int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime);
- /**
- * Allocate a zero-copy options structure.
- *
- * You must free all options structures allocated with this function using
- * hadoopRzOptionsFree.
- *
- * @return A zero-copy options structure, or NULL if one could
- * not be allocated. If NULL is returned, errno will
- * contain the error number.
- */
- struct hadoopRzOptions *hadoopRzOptionsAlloc(void);
- /**
- * Determine whether we should skip checksums in read0.
- *
- * @param opts The options structure.
- * @param skip Nonzero to skip checksums sometimes; zero to always
- * check them.
- *
- * @return 0 on success; -1 plus errno on failure.
- */
- int hadoopRzOptionsSetSkipChecksum(
- struct hadoopRzOptions *opts, int skip);
- /**
- * Set the ByteBufferPool to use with read0.
- *
- * @param opts The options structure.
- * @param className If this is NULL, we will not use any
- * ByteBufferPool. If this is non-NULL, it will be
- * treated as the name of the pool class to use.
- * For example, you can use
- * ELASTIC_BYTE_BUFFER_POOL_CLASS.
- *
- * @return 0 if the ByteBufferPool class was found and
- * instantiated;
- * -1 plus errno otherwise.
- */
- int hadoopRzOptionsSetByteBufferPool(
- struct hadoopRzOptions *opts, const char *className);
- /**
- * Free a hadoopRzOptionsFree structure.
- *
- * @param opts The options structure to free.
- * Any associated ByteBufferPool will also be freed.
- */
- void hadoopRzOptionsFree(struct hadoopRzOptions *opts);
- /**
- * Perform a byte buffer read.
- * If possible, this will be a zero-copy (mmap) read.
- *
- * @param file The file to read from.
- * @param opts An options structure created by hadoopRzOptionsAlloc.
- * @param maxLength The maximum length to read. We may read fewer bytes
- * than this length.
- *
- * @return On success, returns a new hadoopRzBuffer.
- * This buffer will continue to be valid and readable
- * until it is released by readZeroBufferFree. Failure to
- * release a buffer will lead to a memory leak.
- *
- * NULL plus an errno code on an error.
- * errno = EOPNOTSUPP indicates that we could not do a
- * zero-copy read, and there was no ByteBufferPool
- * supplied.
- */
- struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
- struct hadoopRzOptions *opts, int32_t maxLength);
- /**
- * Determine the length of the buffer returned from readZero.
- *
- * @param buffer a buffer returned from readZero.
- * @return the length of the buffer.
- */
- int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer);
- /**
- * Get a pointer to the raw buffer returned from readZero.
- *
- * To find out how many bytes this buffer contains, call
- * hadoopRzBufferLength.
- *
- * @param buffer a buffer returned from readZero.
- * @return a pointer to the start of the buffer. This will be
- * NULL when end-of-file has been reached.
- */
- const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer);
- /**
- * Release a buffer obtained through readZero.
- *
- * @param file The hdfs stream that created this buffer. This must be
- * the same stream you called hadoopReadZero on.
- * @param buffer The buffer to release.
- */
- void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer);
- #ifdef __cplusplus
- }
- #endif
- #endif /*LIBHDFS_HDFS_H*/
- /**
- * vim: ts=4: sw=4: et
- */
|