123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389 |
- /**
- * 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 ZOOKEEPER_H_
- #define ZOOKEEPER_H_
- #include <stdlib.h>
- #include <sys/time.h>
- #include <stdio.h>
- #include "zookeeper_version.h"
- #include "recordio.h"
- #include "zookeeper.jute.h"
- /**
- * \file zookeeper.h
- * \brief ZooKeeper functions and definitions.
- *
- * ZooKeeper is a network service that may be backed by a cluster of
- * synchronized servers. The data in the service is represented as a tree
- * of data nodes. Each node has data, children, an ACL, and status information.
- * The data for a node is read and write in its entirety.
- *
- * ZooKeeper clients can leave watches when they queries the data or children
- * of a node. If a watch is left, that client will be notified of the change.
- * The notification is a one time trigger. Subsequent chances to the node will
- * not trigger a notification unless the client issues a querity with the watch
- * flag set. If the client is ever disconnected from the service, even if the
- * disconnection is temporary, the watches of the client will be removed from
- * the service, so a client must treat a disconnect notification as an implicit
- * trigger of all outstanding watches.
- *
- * When a node is created, it may be flagged as an ephemeral node. Ephemeral
- * nodes are automatically removed when a client session is closed or when
- * a session times out due to inactivity (the ZooKeeper runtime fills in
- * periods of inactivity with pings). Ephemeral nodes cannot have children.
- *
- * ZooKeeper clients are identified by a server assigned session id. For
- * security reasons The server
- * also generates a corresponding password for a session. A client may save its
- * id and corresponding password to persistent storage in order to use the
- * session across program invocation boundaries.
- */
- /* Support for building on various platforms */
- // on cygwin we should take care of exporting/importing symbols properly
- #ifdef DLL_EXPORT
- # define ZOOAPI __declspec(dllexport)
- #else
- # if defined(__CYGWIN__) && !defined(USE_STATIC_LIB)
- # define ZOOAPI __declspec(dllimport)
- # else
- # define ZOOAPI
- # endif
- #endif
- /** zookeeper return constants **/
- enum ZOO_ERRORS {
- ZOK = 0, /*!< Everything is OK */
- /** System and server-side errors.
- * This is never thrown by the server, it shouldn't be used other than
- * to indicate a range. Specifically error codes greater than this
- * value, but lesser than {@link #ZAPIERROR}, are system errors. */
- ZSYSTEMERROR = -1,
- ZRUNTIMEINCONSISTENCY = -2, /*!< A runtime inconsistency was found */
- ZDATAINCONSISTENCY = -3, /*!< A data inconsistency was found */
- ZCONNECTIONLOSS = -4, /*!< Connection to the server has been lost */
- ZMARSHALLINGERROR = -5, /*!< Error while marshalling or unmarshalling data */
- ZUNIMPLEMENTED = -6, /*!< Operation is unimplemented */
- ZOPERATIONTIMEOUT = -7, /*!< Operation timeout */
- ZBADARGUMENTS = -8, /*!< Invalid arguments */
- ZINVALIDSTATE = -9, /*!< Invliad zhandle state */
- /** API errors.
- * This is never thrown by the server, it shouldn't be used other than
- * to indicate a range. Specifically error codes greater than this
- * value are API errors (while values less than this indicate a
- * {@link #ZSYSTEMERROR}).
- */
- ZAPIERROR = -100,
- ZNONODE = -101, /*!< Node does not exist */
- ZNOAUTH = -102, /*!< Not authenticated */
- ZBADVERSION = -103, /*!< Version conflict */
- ZNOCHILDRENFOREPHEMERALS = -108, /*!< Ephemeral nodes may not have children */
- ZNODEEXISTS = -110, /*!< The node already exists */
- ZNOTEMPTY = -111, /*!< The node has children */
- ZSESSIONEXPIRED = -112, /*!< The session has been expired by the server */
- ZINVALIDCALLBACK = -113, /*!< Invalid callback specified */
- ZINVALIDACL = -114, /*!< Invalid ACL specified */
- ZAUTHFAILED = -115, /*!< Client authentication failed */
- ZCLOSING = -116, /*!< ZooKeeper is closing */
- ZNOTHING = -117, /*!< (not error) no server responses to process */
- ZSESSIONMOVED = -118 /*!<session moved to another server, so operation is ignored */
- };
- #ifdef __cplusplus
- extern "C" {
- #endif
- /**
- * @name Debug levels
- */
- typedef enum {ZOO_LOG_LEVEL_ERROR=1,ZOO_LOG_LEVEL_WARN=2,ZOO_LOG_LEVEL_INFO=3,ZOO_LOG_LEVEL_DEBUG=4} ZooLogLevel;
- /**
- * @name ACL Consts
- */
- extern ZOOAPI const int ZOO_PERM_READ;
- extern ZOOAPI const int ZOO_PERM_WRITE;
- extern ZOOAPI const int ZOO_PERM_CREATE;
- extern ZOOAPI const int ZOO_PERM_DELETE;
- extern ZOOAPI const int ZOO_PERM_ADMIN;
- extern ZOOAPI const int ZOO_PERM_ALL;
- /** This Id represents anyone. */
- extern ZOOAPI struct Id ZOO_ANYONE_ID_UNSAFE;
- /** This Id is only usable to set ACLs. It will get substituted with the
- * Id's the client authenticated with.
- */
- extern ZOOAPI struct Id ZOO_AUTH_IDS;
- /** This is a completely open ACL*/
- extern ZOOAPI struct ACL_vector ZOO_OPEN_ACL_UNSAFE;
- /** This ACL gives the world the ability to read. */
- extern ZOOAPI struct ACL_vector ZOO_READ_ACL_UNSAFE;
- /** This ACL gives the creators authentication id's all permissions. */
- extern ZOOAPI struct ACL_vector ZOO_CREATOR_ALL_ACL;
- /**
- * @name Interest Consts
- * These constants are used to express interest in an event and to
- * indicate to zookeeper which events have occurred. They can
- * be ORed together to express multiple interests. These flags are
- * used in the interest and event parameters of
- * \ref zookeeper_interest and \ref zookeeper_process.
- */
- // @{
- extern ZOOAPI const int ZOOKEEPER_WRITE;
- extern ZOOAPI const int ZOOKEEPER_READ;
- // @}
- /**
- * @name Create Flags
- *
- * These flags are used by zoo_create to affect node create. They may
- * be ORed together to combine effects.
- */
- // @{
- extern ZOOAPI const int ZOO_EPHEMERAL;
- extern ZOOAPI const int ZOO_SEQUENCE;
- // @}
- /**
- * @name State Consts
- * These constants represent the states of a zookeeper connection. They are
- * possible parameters of the watcher callback.
- */
- // @{
- extern ZOOAPI const int ZOO_EXPIRED_SESSION_STATE;
- extern ZOOAPI const int ZOO_AUTH_FAILED_STATE;
- extern ZOOAPI const int ZOO_CONNECTING_STATE;
- extern ZOOAPI const int ZOO_ASSOCIATING_STATE;
- extern ZOOAPI const int ZOO_CONNECTED_STATE;
- // @}
- /**
- * @name Watch Types
- * These constants indicate the event that caused the watch event. They are
- * possible values of the first parameter of the watcher callback.
- */
- // @{
- /**
- * \brief a node has been created.
- *
- * This is only generated by watches on non-existent nodes. These watches
- * are set using \ref zoo_exists.
- */
- extern ZOOAPI const int ZOO_CREATED_EVENT;
- /**
- * \brief a node has been deleted.
- *
- * This is only generated by watches on nodes. These watches
- * are set using \ref zoo_exists and \ref zoo_get.
- */
- extern ZOOAPI const int ZOO_DELETED_EVENT;
- /**
- * \brief a node has changed.
- *
- * This is only generated by watches on nodes. These watches
- * are set using \ref zoo_exists and \ref zoo_get.
- */
- extern ZOOAPI const int ZOO_CHANGED_EVENT;
- /**
- * \brief a change as occurred in the list of children.
- *
- * This is only generated by watches on the child list of a node. These watches
- * are set using \ref zoo_get_children or \ref zoo_get_children2.
- */
- extern ZOOAPI const int ZOO_CHILD_EVENT;
- /**
- * \brief a session has been lost.
- *
- * This is generated when a client loses contact or reconnects with a server.
- */
- extern ZOOAPI const int ZOO_SESSION_EVENT;
- /**
- * \brief a watch has been removed.
- *
- * This is generated when the server for some reason, probably a resource
- * constraint, will no longer watch a node for a client.
- */
- extern ZOOAPI const int ZOO_NOTWATCHING_EVENT;
- // @}
- /**
- * \brief ZooKeeper handle.
- *
- * This is the handle that represents a connection to the ZooKeeper service.
- * It is needed to invoke any ZooKeeper function. A handle is obtained using
- * \ref zookeeper_init.
- */
- typedef struct _zhandle zhandle_t;
- /**
- * \brief client id structure.
- *
- * This structure holds the id and password for the session. This structure
- * should be treated as opaque. It is received from the server when a session
- * is established and needs to be sent back as-is when reconnecting a session.
- */
- typedef struct {
- int64_t client_id;
- char passwd[16];
- } clientid_t;
- /**
- * \brief signature of a watch function.
- *
- * There are two ways to receive watch notifications: legacy and watcher object.
- * <p>
- * The legacy style, an application wishing to receive events from ZooKeeper must
- * first implement a function with this signature and pass a pointer to the function
- * to \ref zookeeper_init. Next, the application sets a watch by calling one of
- * the getter API that accept the watch integer flag (for example, \ref zoo_aexists,
- * \ref zoo_get, etc).
- * <p>
- * The watcher object style uses an instance of a "watcher object" which in
- * the C world is represented by a pair: a pointer to a function implementing this
- * signature and a pointer to watcher context -- handback user-specific data.
- * When a watch is triggered this function will be called along with
- * the watcher context. An application wishing to use this style must use
- * the getter API functions with the "w" prefix in their names (for example, \ref
- * zoo_awexists, \ref zoo_wget, etc).
- *
- * \param zh zookeeper handle
- * \param type event type. This is one of the *_EVENT constants.
- * \param state connection state. The state value will be one of the *_STATE constants.
- * \param path znode path for which the watcher is triggered. NULL if the event
- * type is ZOO_SESSION_EVENT
- * \param watcherCtx watcher context.
- */
- typedef void (*watcher_fn)(zhandle_t *zh, int type,
- int state, const char *path,void *watcherCtx);
- /**
- * \brief create a handle to used communicate with zookeeper.
- *
- * This method creates a new handle and a zookeeper session that corresponds
- * to that handle. Session establishment is asynchronous, meaning that the
- * session should not be considered established until (and unless) an
- * event of state ZOO_CONNECTED_STATE is received.
- * \param host comma separated host:port pairs, each corresponding to a zk
- * server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
- * \param fn the global watcher callback function. When notifications are
- * triggered this function will be invoked.
- * \param clientid the id of a previously established session that this
- * client will be reconnecting to. Pass 0 if not reconnecting to a previous
- * session. Clients can access the session id of an established, valid,
- * connection by calling \ref zoo_client_id. If the session corresponding to
- * the specified clientid has expired, or if the clientid is invalid for
- * any reason, the returned zhandle_t will be invalid -- the zhandle_t
- * state will indicate the reason for failure (typically
- * ZOO_EXPIRED_SESSION_STATE).
- * \param context the handback object that will be associated with this instance
- * of zhandle_t. Application can access it (for example, in the watcher
- * callback) using \ref zoo_get_context. The object is not used by zookeeper
- * internally and can be null.
- * \param flags reserved for future use. Should be set to zero.
- * \return a pointer to the opaque zhandle structure. If it fails to create
- * a new zhandle the function returns NULL and the errno variable
- * indicates the reason.
- */
- ZOOAPI zhandle_t *zookeeper_init(const char *host, watcher_fn fn,
- int recv_timeout, const clientid_t *clientid, void *context, int flags);
- /**
- * \brief close the zookeeper handle and free up any resources.
- *
- * After this call, the client session will no longer be valid. The function
- * will flush any outstanding send requests before return. As a result it may
- * block.
- *
- * This method should only be called only once on a zookeeper handle. Calling
- * twice will cause undefined (and probably undesirable behavior). Calling any other
- * zookeeper method after calling close is undefined behaviour and should be avoided.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \return a result code. Regardless of the error code returned, the zhandle
- * will be destroyed and all resources freed.
- *
- * ZOK - success
- * ZBADARGUMENTS - invalid input parameters
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- * ZOPERATIONTIMEOUT - failed to flush the buffers within the specified timeout.
- * ZCONNECTIONLOSS - a network error occured while attempting to send request to server
- * ZSYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details
- */
- ZOOAPI int zookeeper_close(zhandle_t *zh);
- /**
- * \brief return the client session id, only valid if the connections
- * is currently connected (ie. last watcher state is ZOO_CONNECTED_STATE)
- */
- ZOOAPI const clientid_t *zoo_client_id(zhandle_t *zh);
- ZOOAPI int zoo_recv_timeout(zhandle_t *zh);
- ZOOAPI const void *zoo_get_context(zhandle_t *zh);
- ZOOAPI void zoo_set_context(zhandle_t *zh, void *context);
- /**
- * \brief set a watcher function
- * \return previous watcher function
- */
- ZOOAPI watcher_fn zoo_set_watcher(zhandle_t *zh,watcher_fn newFn);
- #ifndef THREADED
- /**
- * \brief Returns the events that zookeeper is interested in.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param fd is the file descriptor of interest
- * \param interest is an or of the ZOOKEEPER_WRITE and ZOOKEEPER_READ flags to
- * indicate the I/O of interest on fd.
- * \param tv a timeout value to be used with select/poll system call
- * \return a result code.
- * ZOK - success
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZCONNECTIONLOSS - a network error occured while attempting to establish
- * a connection to the server
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- * ZOPERATIONTIMEOUT - hasn't received anything from the server for 2/3 of the
- * timeout value specified in zookeeper_init()
- * ZSYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details
- */
- ZOOAPI int zookeeper_interest(zhandle_t *zh, int *fd, int *interest,
- struct timeval *tv);
- /**
- * \brief Notifies zookeeper that an event of interest has happened.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param events will be an OR of the ZOOKEEPER_WRITE and ZOOKEEPER_READ flags.
- * \return a result code.
- * ZOK - success
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZCONNECTIONLOSS - a network error occured while attempting to send request to server
- * ZSESSIONEXPIRED - connection attempt failed -- the session's expired
- * ZAUTHFAILED - authentication request failed, e.i. invalid credentials
- * ZRUNTIMEINCONSISTENCY - a server response came out of order
- * ZSYSTEMERROR -- a system (OS) error occured; it's worth checking errno to get details
- * ZNOTHING -- not an error; simply indicates that there no more data from the server
- * to be processed (when called with ZOOKEEPER_READ flag).
- */
- ZOOAPI int zookeeper_process(zhandle_t *zh, int events);
- #endif
- /**
- * \brief signature of a completion function for a call that returns void.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*void_completion_t)(int rc, const void *data);
- /**
- * \brief signature of a completion function that returns a Stat structure.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param stat a pointer to the stat information for the node involved in
- * this function. If a non zero error code is returned, the content of
- * stat is undefined. The programmer is NOT responsible for freeing stat.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*stat_completion_t)(int rc, const struct Stat *stat,
- const void *data);
- /**
- * \brief signature of a completion function that returns data.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param value the value of the information returned by the asynchronous call.
- * If a non zero error code is returned, the content of value is undefined.
- * The programmer is NOT responsible for freeing value.
- * \param value_len the number of bytes in value.
- * \param stat a pointer to the stat information for the node involved in
- * this function. If a non zero error code is returned, the content of
- * stat is undefined. The programmer is NOT responsible for freeing stat.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*data_completion_t)(int rc, const char *value, int value_len,
- const struct Stat *stat, const void *data);
- /**
- * \brief signature of a completion function that returns a list of strings.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param strings a pointer to the structure containng the list of strings of the
- * names of the children of a node. If a non zero error code is returned,
- * the content of strings is undefined. The programmer is NOT responsible
- * for freeing strings.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*strings_completion_t)(int rc,
- const struct String_vector *strings, const void *data);
- /**
- * \brief signature of a completion function that returns a list of strings and stat.
- * .
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param strings a pointer to the structure containng the list of strings of the
- * names of the children of a node. If a non zero error code is returned,
- * the content of strings is undefined. The programmer is NOT responsible
- * for freeing strings.
- * \param stat a pointer to the stat information for the node involved in
- * this function. If a non zero error code is returned, the content of
- * stat is undefined. The programmer is NOT responsible for freeing stat.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*strings_stat_completion_t)(int rc,
- const struct String_vector *strings, const struct Stat *stat,
- const void *data);
- /**
- * \brief signature of a completion function that returns a list of strings.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param value the value of the string returned.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void
- (*string_completion_t)(int rc, const char *value, const void *data);
- /**
- * \brief signature of a completion function that returns an ACL.
- *
- * This method will be invoked at the end of a asynchronous call and also as
- * a result of connection loss or timeout.
- * \param rc the error code of the call. Connection loss/timeout triggers
- * the completion with one of the following error codes:
- * ZCONNECTIONLOSS -- lost connection to the server
- * ZOPERATIONTIMEOUT -- connection timed out
- * Data related events trigger the completion with error codes listed the
- * Exceptions section of the documentation of the function that initiated the
- * call. (Zero indicates call was successful.)
- * \param acl a pointer to the structure containng the ACL of a node. If a non
- * zero error code is returned, the content of strings is undefined. The
- * programmer is NOT responsible for freeing acl.
- * \param stat a pointer to the stat information for the node involved in
- * this function. If a non zero error code is returned, the content of
- * stat is undefined. The programmer is NOT responsible for freeing stat.
- * \param data the pointer that was passed by the caller when the function
- * that this completion corresponds to was invoked. The programmer
- * is responsible for any memory freeing associated with the data
- * pointer.
- */
- typedef void (*acl_completion_t)(int rc, struct ACL_vector *acl,
- struct Stat *stat, const void *data);
- /**
- * \brief get the state of the zookeeper connection.
- *
- * The return value will be one of the \ref State Consts.
- */
- ZOOAPI int zoo_state(zhandle_t *zh);
- /**
- * \brief create a node.
- *
- * This method will create a node in ZooKeeper. A node can only be created if
- * it does not already exists. The Create Flags affect the creation of nodes.
- * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the
- * client session goes away. If the ZOO_SEQUENCE flag is set, a unique
- * monotonically increasing sequence number is appended to the path name.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path The name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param value The data to be stored in the node.
- * \param valuelen The number of bytes in data.
- * \param acl The initial ACL of the node. If null, the ACL of the parent will be
- * used.
- * \param flags this parameter can be set to 0 for normal create or an OR
- * of the Create Flags
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the parent node does not exist.
- * ZNODEEXISTS the node already exists
- * ZNOAUTH the client does not have permission.
- * ZNOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.
- * \param data The data that will be passed to the completion routine when the
- * function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_acreate(zhandle_t *zh, const char *path, const char *value,
- int valuelen, const struct ACL_vector *acl, int flags,
- string_completion_t completion, const void *data);
- /**
- * \brief delete a node in zookeeper.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param version the expected version of the node. The function will fail if the
- * actual version of the node does not match the expected version.
- * If -1 is used the version check will not take place.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADVERSION expected version does not match actual version.
- * ZNOTEMPTY children are present; node cannot be deleted.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_adelete(zhandle_t *zh, const char *path, int version,
- void_completion_t completion, const void *data);
- /**
- * \brief checks the existence of a node in zookeeper.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify the
- * client if the node changes. The watch will be set even if the node does not
- * exist. This allows clients to watch for nodes to appear.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when the
- * function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch,
- stat_completion_t completion, const void *data);
- /**
- * \brief checks the existence of a node in zookeeper.
- *
- * This function is similar to \ref zoo_axists except it allows one specify
- * a watcher object - a function pointer and associated context. The function
- * will be called once the watch has fired. The associated context data will be
- * passed to the function as the watcher context parameter.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null a watch will set on the specified znode on the server.
- * The watch will be set even if the node does not exist. This allows clients
- * to watch for nodes to appear.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when the
- * function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_awexists(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- stat_completion_t completion, const void *data);
- /**
- * \brief gets the data associated with a node.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch,
- data_completion_t completion, const void *data);
- /**
- * \brief gets the data associated with a node.
- *
- * This function is similar to \ref zoo_aget except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_awget(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- data_completion_t completion, const void *data);
- /**
- * \brief sets the data associated with a node.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param buffer the buffer holding data to be written to the node.
- * \param buflen the number of bytes from buffer to write.
- * \param version the expected version of the node. The function will fail if
- * the actual version of the node does not match the expected version. If -1 is
- * used the version check will not take place. * completion: If null,
- * the function will execute synchronously. Otherwise, the function will return
- * immediately and invoke the completion routine when the request completes.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADVERSION expected version does not match actual version.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen,
- int version, stat_completion_t completion, const void *data);
- /**
- * \brief lists the children of a node.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
- strings_completion_t completion, const void *data);
- /**
- * \brief lists the children of a node.
- *
- * This function is similar to \ref zoo_aget_children except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_awget_children(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- strings_completion_t completion, const void *data);
- /**
- * \brief lists the children of a node, and get the parent stat.
- *
- * This function is new in version 3.3.0
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch,
- strings_stat_completion_t completion, const void *data);
- /**
- * \brief lists the children of a node, and get the parent stat.
- *
- * This function is similar to \ref zoo_aget_children2 except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * This function is new in version 3.3.0
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_awget_children2(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- strings_stat_completion_t completion, const void *data);
- /**
- * \brief Flush leader channel.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_async(zhandle_t *zh, const char *path,
- string_completion_t completion, const void *data);
- /**
- * \brief gets the acl associated with a node.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t completion,
- const void *data);
- /**
- * \brief sets the acl associated with a node.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param buffer the buffer holding the acls to be written to the node.
- * \param buflen the number of bytes from buffer to write.
- * \param completion the routine to invoke when the request completes. The completion
- * will be triggered with one of the following codes passed in as the rc argument:
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZINVALIDACL invalid ACL specified
- * ZBADVERSION expected version does not match actual version.
- * \param data the data that will be passed to the completion routine when
- * the function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_aset_acl(zhandle_t *zh, const char *path, int version,
- struct ACL_vector *acl, void_completion_t, const void *data);
- /**
- * \brief return an error string.
- *
- * \param return code
- * \return string corresponding to the return code
- */
- ZOOAPI const char* zerror(int c);
- /**
- * \brief specify application credentials.
- *
- * The application calls this function to specify its credentials for purposes
- * of authentication. The server will use the security provider specified by
- * the scheme parameter to authenticate the client connection. If the
- * authentication request has failed:
- * - the server connection is dropped
- * - the watcher is called with the ZOO_AUTH_FAILED_STATE value as the state
- * parameter.
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param scheme the id of authentication scheme. Natively supported:
- * "digest" password-based authentication
- * \param cert application credentials. The actual value depends on the scheme.
- * \param certLen the length of the data parameter
- * \param completion the routine to invoke when the request completes. One of
- * the following result codes may be passed into the completion callback:
- * ZOK operation completed successfully
- * ZAUTHFAILED authentication failed
- * \param data the data that will be passed to the completion routine when the
- * function completes.
- * \return ZOK on success or one of the following errcodes on failure:
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- * ZSYSTEMERROR - a system error occured
- */
- ZOOAPI int zoo_add_auth(zhandle_t *zh,const char* scheme,const char* cert,
- int certLen, void_completion_t completion, const void *data);
- /**
- * \brief checks if the current zookeeper connection state can't be recovered.
- *
- * The application must close the zhandle and try to reconnect.
- *
- * \param zh the zookeeper handle (see \ref zookeeper_init)
- * \return ZINVALIDSTATE if connection is unrecoverable
- */
- ZOOAPI int is_unrecoverable(zhandle_t *zh);
- /**
- * \brief sets the debugging level for the library
- */
- ZOOAPI void zoo_set_debug_level(ZooLogLevel logLevel);
- /**
- * \brief sets the stream to be used by the library for logging
- *
- * The zookeeper library uses stderr as its default log stream. Application
- * must make sure the stream is writable. Passing in NULL resets the stream
- * to its default value (stderr).
- */
- ZOOAPI void zoo_set_log_stream(FILE* logStream);
- /**
- * \brief enable/disable quorum endpoint order randomization
- *
- * If passed a non-zero value, will make the client connect to quorum peers
- * in the order as specified in the zookeeper_init() call.
- * A zero value causes zookeeper_init() to permute the peer endpoints
- * which is good for more even client connection distribution among the
- * quorum peers.
- */
- ZOOAPI void zoo_deterministic_conn_order(int yesOrNo);
- /**
- * \brief create a node synchronously.
- *
- * This method will create a node in ZooKeeper. A node can only be created if
- * it does not already exists. The Create Flags affect the creation of nodes.
- * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the
- * client session goes away. If the ZOO_SEQUENCE flag is set, a unique
- * monotonically increasing sequence number is appended to the path name.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path The name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param value The data to be stored in the node.
- * \param valuelen The number of bytes in data. To set the data to be NULL use
- * value as NULL and valuelen as -1.
- * \param acl The initial ACL of the node. If null, the ACL of the parent will be
- * used.
- * \param flags this parameter can be set to 0 for normal create or an OR
- * of the Create Flags
- * \param path_buffer Buffer which will be filled with the path of the
- * new node (this might be different than the supplied path
- * because of the ZOO_SEQUENCE flag). The path string will always be
- * null-terminated.
- * \param path_buffer_len Size of path buffer; if the path of the new
- * node (including space for the null terminator) exceeds the buffer size,
- * the path string will be truncated to fit. The actual path of the
- * new node in the server will not be affected by the truncation.
- * The path string will always be null-terminated.
- * \return one of the following codes are returned:
- * ZOK operation completed succesfully
- * ZNONODE the parent node does not exist.
- * ZNODEEXISTS the node already exists
- * ZNOAUTH the client does not have permission.
- * ZNOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
- int valuelen, const struct ACL_vector *acl, int flags,
- char *path_buffer, int path_buffer_len);
- /**
- * \brief delete a node in zookeeper synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param version the expected version of the node. The function will fail if the
- * actual version of the node does not match the expected version.
- * If -1 is used the version check will not take place.
- * \return one of the following values is returned.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADVERSION expected version does not match actual version.
- * ZNOTEMPTY children are present; node cannot be deleted.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_delete(zhandle_t *zh, const char *path, int version);
- /**
- * \brief checks the existence of a node in zookeeper synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify the
- * client if the node changes. The watch will be set even if the node does not
- * exist. This allows clients to watch for nodes to appear.
- * \param the return stat value of the node.
- * \return return code of the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat);
- /**
- * \brief checks the existence of a node in zookeeper synchronously.
- *
- * This function is similar to \ref zoo_exists except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null a watch will set on the specified znode on the server.
- * The watch will be set even if the node does not exist. This allows clients
- * to watch for nodes to appear.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param the return stat value of the node.
- * \return return code of the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_wexists(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx, struct Stat *stat);
- /**
- * \brief gets the data associated with a node synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param buffer the buffer holding the node data returned by the server
- * \param buffer_len is the size of the buffer pointed to by the buffer parameter.
- * It'll be set to the actual data length upon return. If the data is NULL, length is -1.
- * \param stat if not NULL, will hold the value of stat for the path on return.
- * \return return value of the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,
- int* buffer_len, struct Stat *stat);
- /**
- * \brief gets the data associated with a node synchronously.
- *
- * This function is similar to \ref zoo_get except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param buffer the buffer holding the node data returned by the server
- * \param buffer_len is the size of the buffer pointed to by the buffer parameter.
- * It'll be set to the actual data length upon return. If the data is NULL, length is -1.
- * \param stat if not NULL, will hold the value of stat for the path on return.
- * \return return value of the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either in ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_wget(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- char *buffer, int* buffer_len, struct Stat *stat);
- /**
- * \brief sets the data associated with a node. See zoo_set2 function if
- * you require access to the stat information associated with the znode.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param buffer the buffer holding data to be written to the node.
- * \param buflen the number of bytes from buffer to write. To set NULL as data
- * use buffer as NULL and buflen as -1.
- * \param version the expected version of the node. The function will fail if
- * the actual version of the node does not match the expected version. If -1 is
- * used the version check will not take place.
- * \return the return code for the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADVERSION expected version does not match actual version.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_set(zhandle_t *zh, const char *path, const char *buffer,
- int buflen, int version);
- /**
- * \brief sets the data associated with a node. This function is the same
- * as zoo_set except that it also provides access to stat information
- * associated with the znode.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param buffer the buffer holding data to be written to the node.
- * \param buflen the number of bytes from buffer to write. To set NULL as data
- * use buffer as NULL and buflen as -1.
- * \param version the expected version of the node. The function will fail if
- * the actual version of the node does not match the expected version. If -1 is
- * used the version check will not take place.
- * \param stat if not NULL, will hold the value of stat for the path on return.
- * \return the return code for the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADVERSION expected version does not match actual version.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_set2(zhandle_t *zh, const char *path, const char *buffer,
- int buflen, int version, struct Stat *stat);
- /**
- * \brief lists the children of a node synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param strings return value of children paths.
- * \return the return code of the function.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch,
- struct String_vector *strings);
- /**
- * \brief lists the children of a node synchronously.
- *
- * This function is similar to \ref zoo_get_children except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param strings return value of children paths.
- * \return the return code of the function.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_wget_children(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- struct String_vector *strings);
- /**
- * \brief lists the children of a node and get its stat synchronously.
- *
- * This function is new in version 3.3.0
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watch if nonzero, a watch will be set at the server to notify
- * the client if the node changes.
- * \param strings return value of children paths.
- * \param stat return value of node stat.
- * \return the return code of the function.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_get_children2(zhandle_t *zh, const char *path, int watch,
- struct String_vector *strings, struct Stat *stat);
- /**
- * \brief lists the children of a node and get its stat synchronously.
- *
- * This function is similar to \ref zoo_get_children except it allows one specify
- * a watcher object rather than a boolean watch flag.
- *
- * This function is new in version 3.3.0
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param watcher if non-null, a watch will be set at the server to notify
- * the client if the node changes.
- * \param watcherCtx user specific data, will be passed to the watcher callback.
- * Unlike the global context set by \ref zookeeper_init, this watcher context
- * is associated with the given instance of the watcher only.
- * \param strings return value of children paths.
- * \param stat return value of node stat.
- * \return the return code of the function.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_wget_children2(zhandle_t *zh, const char *path,
- watcher_fn watcher, void* watcherCtx,
- struct String_vector *strings, struct Stat *stat);
- /**
- * \brief gets the acl associated with a node synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param acl the return value of acls on the path.
- * \param stat returns the stat of the path specified.
- * \return the return code for the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl,
- struct Stat *stat);
- /**
- * \brief sets the acl associated with a node synchronously.
- *
- * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
- * \param path the name of the node. Expressed as a file name with slashes
- * separating ancestors of the node.
- * \param version the expected version of the path.
- * \param acl the acl to be set on the path.
- * \return the return code for the function call.
- * ZOK operation completed succesfully
- * ZNONODE the node does not exist.
- * ZNOAUTH the client does not have permission.
- * ZINVALIDACL invalid ACL specified
- * ZBADVERSION expected version does not match actual version.
- * ZBADARGUMENTS - invalid input parameters
- * ZINVALIDSTATE - zhandle state is either ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
- * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
- */
- ZOOAPI int zoo_set_acl(zhandle_t *zh, const char *path, int version,
- const struct ACL_vector *acl);
- #ifdef __cplusplus
- }
- #endif
- #endif /*ZOOKEEPER_H_*/
|