Author: mahadev
Date: Thu May 14 06:37:59 2009
New Revision: 774656
URL: http://svn.apache.org/viewvc?rev=774656&view=rev
Log:
ZOOKEEPER-395. Python bindings. (henry robinson via mahadev)
Added:
hadoop/zookeeper/trunk/src/contrib/zkpython/
hadoop/zookeeper/trunk/src/contrib/zkpython/README
hadoop/zookeeper/trunk/src/contrib/zkpython/build.xml
hadoop/zookeeper/trunk/src/contrib/zkpython/src/
hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/
hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/zookeeper.c
hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/
hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/setup.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/zk.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/connection_test.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/create_test.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/exists_test.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/get_set_test.py
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/run_tests.sh
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zkServer.sh
hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zktestbase.py
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=774656&r1=774655&r2=774656&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu May 14 06:37:59 2009
@@ -141,6 +141,8 @@
ZOOKEEPER-36. REST access to ZooKeeper (phunt via mahadev)
+ ZOOKEEPER-395. Python bindings. (henry robinson via mahadev)
+
Release 3.1.0 - 2009-02-06
Non-backward compatible changes:
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/README
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/README?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/README (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/README Thu May 14 06:37:59 2009
@@ -0,0 +1,83 @@
+Very early version of ZooKeeper bindings for Python. All functions are imported as methods into the zookeeper module.
+
+DEPENDENCIES:
+-------------
+
+This has only been tested against SVN (i.e. 3.2.0 in development) but should work against 3.1.1.
+
+You will need the Python development headers installed to build the module - on many package-management systems, these can be found in python-devel.
+
+Python >= 2.3 is required. We have tested against 2.3, 2.5 and 2.6. We have not tested against 3.x.
+
+BUILD AND INSTALL:
+-------------------
+
+To install, make sure that the C client has been built and that the libraries are installed in /usr/local/lib (or change this directory in setup.py). Then run:
+
+ant install
+
+from zookeeper/src/contrib/zkpython/.
+
+To test, run ant test from the same directory.
+
+NAMING CONVENTIONS:
+--------------------
+
+All methods that in the C library are zoo_fn_name have been implemented as zookeeper.fn_name. The exception is any function that has a watch function argument is named without the 'w' prefix (for example, zoo_wexists becomes zookeeper.exists). The variants of these functions without the watch argument (i.e. zoo_exists) have not been implemented on the understanding that they are superceded by the zoo_w* API.
+
+Enums and integer constants that begin ZOO_int_name are named as zookeeper.int_name.
+
+PARAMETER CHANGES:
+------------------
+
+Zookeeper handles are represented as integers to avoid marshalling the entire structure for every call. Therefore they are opaque from Python.
+
+Any parameter that is used to provide arguments to callback methods is not exposed in the API. Python provides better mechanisms for providing a closure to be called in the future.
+
+Every callback gets passed the handle of the ZooKeeper instance used to register the callback.
+
+DATA TYPES:
+-----------
+
+ACL_vectors are lists of dictionaries. Stat structures are dictionaries. String_vectors are lists of strings.
+
+EXCEPTIONS AND ERROR HANDLING:
+------------------------------
+
+Currently synchronous calls indicate failure by throwing an exception (note that this includes the synchronous calls to set up asynchronous completion callbacks!). Success is returned as an integer.
+
+Callbacks signify failure by having the integer response code passed in.
+
+WHAT'S NEW IN 0.3:
+------------------
+
+Some tests in zkpython/test. More to follow!
+
+A variety of bugfixes.
+
+Changed the way methods return results - all responses are integers now, for the client to convert to a string if it needs.
+
+WHAT'S NEW IN 0.2:
+------------------
+
+The asynchronous API is now implemented (see zookeeper.a*).
+
+Most enums defined in zookeeper.h are now added as constants to the module.
+
+_set2 and a few other edge API calls have been implemented. The module is now nearly 100% feature complete!
+
+A reference count error was tracked down and killed. More probably lurk in there!
+
+WHAT'S NOT DONE / KNOWN ISSUES:
+-------------------------------
+
+1. There may well be memory leaks / reference count issues; I need to do a complete audit.
+2. There probably needs to be a more Pythonic Python-side wrapper for these functions (e.g. a zookeeper object, the ability to iterate through a tree of zk nodes)
+3. set_watcher is not implemented.
+4. If you pass an invalid handle, the library will do anything from silent failure to segfaulting. This will be fixed and an exception will be raised in future versions.
+5. More tests aren't written yet, but should be.
+6. Similarly for documentation, other than this README (in particular, docstrings)
+
+Bug reports / comments very welcome!
+
+Henry Robinson henry@cloudera.com
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/build.xml?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/build.xml (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/build.xml Thu May 14 06:37:59 2009
@@ -0,0 +1,82 @@
+<?xml version="1.0"?>
+
+<!--
+ 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.
+-->
+
+<project name="zkpython" default="install">
+ <import file="../build-contrib.xml"/>
+ <property name="python.src.dir" value="src/python"/>
+ <property name="test.build.dir" value="build/test/" />
+ <property name="test.src.dir" value="src/test"/>
+ <property name="test.log.dir" value="${test.build.dir}/logs" />
+ <property name="test.output" value="no" />
+ <property name="test.timeout" value="900000" />
+
+ <available classname="org.apache.zookeeper.ZooKeeperMain"
+ classpathref="classpath"
+ property="mainIsCompiled"/>
+
+ <target name="checkMainCompiled" unless="mainIsCompiled">
+ <fail message="ZooKeeper main must first be compiled (toplevel build.xml)"/>
+ </target>
+
+
+ <target name="test"
+ depends="test-init,test-category,test-start,python-test,test-stop" />
+
+ <target name="test-init" depends="checkMainCompiled">
+ <delete dir="${test.log.dir}" />
+ <mkdir dir="${test.log.dir}" />
+ </target>
+
+ <target name="test-start">
+ <exec executable="${test.src.dir}/zkServer.sh" failonerror="true">
+ <arg value="startClean"/>
+ </exec>
+ </target>
+
+ <target name="test-stop">
+ <exec executable="${test.src.dir}/zkServer.sh" failonerror="true">
+ <arg value="stop"/>
+ </exec>
+ </target>
+
+ <target name="test-category">
+ <property name="test.category" value=""/>
+ </target>
+
+ <target name="python-test">
+ <exec executable="${test.src.dir}/run_tests.sh" failonerror="true">
+ <arg value="${test.src.dir}"/>
+ </exec>
+ </target>
+
+ <target name="build">
+ <exec executable="python" failonerror="true">
+ <arg value="${python.src.dir}/setup.py"/>
+ <arg value="build"/>
+ </exec>
+ </target>
+
+ <target name="install" depends="build">
+ <exec executable="python" failonerror="true">
+ <arg value="${python.src.dir}/setup.py"/>
+ <arg value="install"/>
+ </exec>
+ </target>
+</project>
+
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/zookeeper.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/zookeeper.c?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/zookeeper.c (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/c/zookeeper.c Thu May 14 06:37:59 2009
@@ -0,0 +1,969 @@
+/**
+ * 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.
+ */
+
+#include <Python.h>
+#include <zookeeper.h>
+#include <assert.h>
+
+#define MAX_ZHANDLES 256
+static zhandle_t* zhandles[MAX_ZHANDLES];
+static int num_zhandles = 0;
+
+/////////////////////////////////////////////
+// HELPER FUNCTIONS
+typedef struct {
+ int zhandle;
+ PyObject *callback;
+}pywatcher_t;
+
+pywatcher_t *create_pywatcher(int zh, PyObject* cb)
+{
+ pywatcher_t *ret = (pywatcher_t*)calloc(sizeof(pywatcher_t),1);
+ Py_INCREF(cb);
+ ret->zhandle = zh; ret->callback = cb;
+ return ret;
+}
+
+void free_pywatcher( pywatcher_t *pw)
+{
+ Py_DECREF(pw->callback);
+ free(pw);
+}
+
+
+PyObject *build_stat( const struct Stat *stat )
+{
+ return Py_BuildValue( "{s:K, s:K, s:K, s:K,"
+ "s:i, s:i, s:i, s:K,"
+ "s:i, s:i, s:K}",
+ "czxid", stat->czxid,
+ "mzxid", stat->mzxid,
+ "ctime", stat->ctime,
+ "mtime", stat->mtime,
+ "version", stat->version,
+ "cversion", stat->cversion,
+ "aversion", stat->aversion,
+ "ephemeralOwner", stat->ephemeralOwner,
+ "dataLength", stat->dataLength,
+ "numChildren", stat->numChildren,
+ "pzxid", stat->pzxid );
+}
+
+PyObject *build_string_vector(const struct String_vector *sv)
+{
+ PyObject *ret = PyList_New(sv->count);
+ int i;
+ for (i=0;i<sv->count;++i)
+ {
+ PyObject *s = PyString_FromString(sv->data[i]);
+ PyList_SetItem(ret, i, s);
+ }
+ return ret;
+}
+
+PyObject *build_acls( const struct ACL_vector *acls )
+{
+ PyObject *ret = PyList_New( acls->count );
+ int i;
+ for (i=0;i<acls->count;++i)
+ {
+ PyObject *acl = Py_BuildValue( "{s:i, s:s, s:s}",
+ "perms", acls->data[i].perms,
+ "scheme", acls->data[i].id.scheme,
+ "id", acls->data[i].id.id );
+ PyList_SetItem(ret, i, acl);
+ }
+ return ret;
+}
+
+void parse_acls( struct ACL_vector *acls, PyObject *pyacls )
+{
+ PyObject *a;
+ acls->count = PyList_Size( pyacls );
+ acls->data = (struct ACL *)calloc( acls->count, sizeof(struct ACL) );
+ int i;
+ for (i=0;i<acls->count;++i)
+ {
+ a = PyList_GetItem( pyacls, i );
+ // a is now a dictionary
+ PyObject *perms = PyDict_GetItemString( a, "perms" );
+ acls->data[i].perms = (int32_t)(PyInt_AsLong(perms));
+ acls->data[i].id.id = strdup( PyString_AsString( PyDict_GetItemString( a, "id" ) ) );
+ acls->data[i].id.scheme = strdup( PyString_AsString( PyDict_GetItemString( a, "scheme" ) ) );
+ }
+}
+
+void free_acls( struct ACL_vector *acls )
+{
+ int i;
+ for (i=0;i<acls->count;++i)
+ {
+ free(acls->data[i].id.id);
+ free(acls->data[i].id.scheme);
+ }
+ free(acls->data);
+}
+
+/////////////////////////////////////////////
+/* Every watcher invocation goes through this dispatch point, which
+ a) acquires the global interpreter lock
+ b) unpacks the PyObject to call from the passed context pointer, which
+ handily includes the index of the relevant zookeeper handle to pass back to Python.
+ c) Makes the call into Python, checking for error conditions which we are responsible fro
+ detecting and doing something about (we just print the error and plough right on)
+ d) releases the lock after freeing up the context object, which is only used for one
+ watch invocation (watches are one-shot)
+*/
+void watcher_dispatch(zhandle_t *zzh, int type, int state, const char *path, void *context)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)context;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,i,s)", pyw->zhandle,type, state, path);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+static PyObject *pyzookeeper_init(PyObject *self, PyObject *args)
+{
+ // TODO: Unpack clientid
+ const char *host;
+ PyObject *watcherfn;
+ int recv_timeout;
+ PyObject *clientid;
+
+ if (!PyArg_ParseTuple(args, "sOii", &host, &watcherfn, &recv_timeout, &clientid))
+ return NULL;
+
+ zhandle_t *zh = zookeeper_init( host, watcherfn != Py_None ? watcher_dispatch : NULL,
+ recv_timeout, NULL,
+ create_pywatcher(num_zhandles,watcherfn), 0 );
+
+ if (zh == NULL)
+ {
+ PyErr_SetString( PyExc_IOError, "Unknown error" );
+ return NULL;
+ }
+
+ zhandles[num_zhandles] = zh;
+ return Py_BuildValue( "i", num_zhandles++ );
+}
+
+///////////////////////////////////////////////////////
+// Similar kind of mechanisms for various completion
+// types
+
+void void_completion_dispatch(int rc, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i)", pyw->zhandle, rc);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+void stat_completion_dispatch(int rc, const struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,O)", pyw->zhandle,rc, build_stat(stat));
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+void data_completion_dispatch(int rc, const char *value, int value_len, const struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,s#,O)", pyw->zhandle,rc, value,value_len, build_stat(stat));
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+void strings_completion_dispatch(int rc, const struct String_vector *strings, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,O)", pyw->zhandle,rc, build_string_vector(strings));
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+void string_completion_dispatch(int rc, const char *value, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,s)", pyw->zhandle,rc, value);
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+void acl_completion_dispatch(int rc, struct ACL_vector *acl, struct Stat *stat, const void *data)
+{
+ PyGILState_STATE gstate;
+ pywatcher_t *pyw = (pywatcher_t*)data;
+ PyObject *callback = pyw->callback;
+ gstate = PyGILState_Ensure();
+ PyObject *arglist = Py_BuildValue("(i,i,O,O)", pyw->zhandle,rc, build_acls(acl), build_stat(stat));
+ if (PyObject_CallObject((PyObject*)callback, arglist) == NULL)
+ PyErr_Print();
+ free_pywatcher(pyw);
+ PyGILState_Release(gstate);
+}
+
+///////////////////////////////////////////////////////
+// Asynchronous API
+
+PyObject *pyzoo_acreate(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; char *value; int valuelen;
+ struct ACL_vector acl; int flags;
+ PyObject *completion_callback;
+ PyObject *pyacls;
+ if (!PyArg_ParseTuple(args, "iss#OiO", &zkhid, &path, &value, &valuelen, &pyacls, &flags, &completion_callback))
+ return NULL;
+
+ parse_acls(&acl, pyacls);
+ int err = zoo_acreate( zhandles[zkhid],
+ path,
+ value,
+ valuelen,
+ &acl,
+ flags,
+ completion_callback != Py_None ? string_completion_dispatch : NULL,
+ completion_callback != Py_None ? create_pywatcher(zkhid, completion_callback ) : NULL );
+
+ free_acls(&acl);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_adelete(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; int version;
+ PyObject *completion_callback;
+ if (!PyArg_ParseTuple(args, "isiO", &zkhid, &path, &version, &completion_callback))
+ return NULL;
+
+ int err = zoo_adelete( zhandles[zkhid],
+ path,
+ version,
+ completion_callback != Py_None ? void_completion_dispatch : NULL,
+ completion_callback != Py_None ? create_pywatcher(zkhid,
+ completion_callback ) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_aexists(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ PyObject *exists_watch;
+ if (!PyArg_ParseTuple(args, "isOO", &zkhid, &path,
+ &exists_watch, &completion_callback))
+ return NULL;
+ int err = zoo_awexists( zhandles[zkhid],
+ path,
+ exists_watch != Py_None ? watcher_dispatch : NULL,
+ exists_watch != Py_None ? create_pywatcher(zkhid, exists_watch) : NULL,
+ (completion_callback != Py_None) ? stat_completion_dispatch : NULL,
+ (completion_callback != Py_None) ? create_pywatcher(zkhid, completion_callback) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_aget(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ PyObject *get_watch;
+ if (!PyArg_ParseTuple(args, "isOO", &zkhid, &path,
+ &get_watch, &completion_callback))
+ return NULL;
+
+ int err = zoo_awget( zhandles[zkhid],
+ path,
+ get_watch != Py_None ? watcher_dispatch : NULL,
+ get_watch != Py_None ? create_pywatcher(zkhid, get_watch) : NULL,
+ completion_callback != Py_None ? data_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+PyObject *pyzoo_aset(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; char *buffer; int buflen; int version;
+ PyObject *completion_callback;
+ if (!PyArg_ParseTuple(args, "iss#iO", &zkhid, &path, &buffer, &buflen, &version, &completion_callback))
+ return NULL;
+
+ int err = zoo_aset( zhandles[zkhid],
+ path,
+ buffer,
+ buflen,
+ version,
+ completion_callback != Py_None ? stat_completion_dispatch : NULL,
+ completion_callback != Py_None ? create_pywatcher(zkhid,
+ completion_callback ) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);
+}
+
+PyObject *pyzoo_aget_children(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ PyObject *get_watch;
+ if (!PyArg_ParseTuple(args, "isOO", &zkhid, &path,
+ &get_watch, &completion_callback))
+ return NULL;
+
+ int err = zoo_awget_children( zhandles[zkhid],
+ path,
+ get_watch != Py_None ? watcher_dispatch : NULL,
+ get_watch != Py_None ? create_pywatcher(zkhid, get_watch) : NULL,
+ completion_callback != Py_None ? strings_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_async(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ if (!PyArg_ParseTuple(args, "isO", &zkhid, &path,
+ &completion_callback))
+ return NULL;
+
+ int err = zoo_async( zhandles[zkhid],
+ path,
+ completion_callback != Py_None ? string_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_aget_acl(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ if (!PyArg_ParseTuple(args, "isiOO", &zkhid, &path,
+ &completion_callback))
+ return NULL;
+
+ int err = zoo_aget_acl( zhandles[zkhid],
+ path,
+ completion_callback != Py_None ? acl_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+PyObject *pyzoo_aset_acl(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; int version;
+ PyObject *completion_callback, *pyacl;
+ struct ACL_vector aclv;
+ if (!PyArg_ParseTuple(args, "isiOO", &zkhid, &path, &version,
+ &pyacl, &completion_callback))
+ return NULL;
+ parse_acls( &aclv, pyacl );
+ int err = zoo_aset_acl( zhandles[zkhid],
+ path,
+ version,
+ &aclv,
+ completion_callback != Py_None ? void_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+ free_acls(&aclv);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+/*PyObject *pyzoo_aget(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path;
+ PyObject *completion_callback;
+ PyObject *get_watch;
+ if (!PyArg_ParseTuple(args, "isiOO", &zkhid, &path,
+ &get_watch, &completion_callback))
+ return NULL;
+
+ int err = zoo_awget( zhandles[zkhid],
+ path,
+ get_watch != Py_None ? watcher_dispatch : NULL,
+ get_watch != Py_None ? create_pywatcher(zkhid, get_watch) : NULL,
+ completion_callback != Py_None ? void_completion_dispatch : NULL,
+ completion_callback != Py_None ?
+ create_pywatcher(zkhid, completion_callback ) : NULL );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+ }*/
+
+PyObject *pyzoo_add_auth(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *scheme, *cert;
+ int certLen;
+ PyObject *completion_callback;
+
+ if (!PyArg_ParseTuple(args, "iss#O", &zkhid, &scheme, &cert, &certLen, &completion_callback))
+ return NULL;
+
+ int err = zoo_add_auth( zhandles[zkhid],
+ scheme,
+ cert,
+ certLen,
+ completion_callback != Py_None ? void_completion_dispatch : NULL,
+ completion_callback != Py_None ? create_pywatcher(zkhid, completion_callback) : NULL );
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+///////////////////////////////////////////////////////
+// synchronous API
+
+static PyObject *pyzoo_create(PyObject *self, PyObject *args)
+{
+ char *path;
+ int zkhid;
+ char* values;
+ int valuelen;
+ PyObject *acl = NULL;
+ int flags = 0;
+ char realbuf[256];
+ const int maxbuf_len = 256;
+ if (!PyArg_ParseTuple(args, "iss#Oi",&zkhid, &path, &values, &valuelen,&acl,&flags))
+ return NULL;
+ struct ACL_vector aclv;
+ parse_acls(&aclv,acl);
+ zhandle_t *zh = zhandles[zkhid];
+ int err = zoo_create(zh, path, values, valuelen, &aclv, flags, realbuf, maxbuf_len);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+
+ return Py_BuildValue("i", err);;
+}
+
+static PyObject *pyzoo_delete(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ int version;
+ if (!PyArg_ParseTuple(args, "isi",&zkhid,&path,&version))
+ return NULL;
+ zhandle_t *zh = zhandles[zkhid];
+ int err = zoo_delete(zh, path, version);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err));
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+static PyObject *pyzoo_exists(PyObject *self, PyObject *args)
+{
+ int zkhid; char *path; PyObject *watcherfn = NULL;
+ struct Stat stat;
+ if (!PyArg_ParseTuple(args, "isO", &zkhid, &path, &watcherfn))
+ return NULL;
+ zhandle_t *zh = zhandles[zkhid];
+ pywatcher_t *pw = NULL;
+ if (watcherfn != Py_None)
+ pw = create_pywatcher(zkhid, watcherfn);
+ int err = zoo_wexists(zh, path, watcherfn != Py_None ? watcher_dispatch : NULL, pw, &stat);
+ if (err != ZOK && err != ZNONODE)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ free_pywatcher( pw );
+ return NULL;
+ }
+ if (err == ZOK)
+ return build_stat( &stat );
+ if (err == ZNONODE)
+ return Py_None; // This isn't exceptional
+ return NULL;
+}
+
+static PyObject *pyzoo_get_children(PyObject *self, PyObject *args)
+{
+ // TO DO: Does Python copy the string or the reference? If it's the former
+ // we should free the String_vector
+ int zkhid;
+ char *path;
+ PyObject *watcherfn = NULL;
+ struct String_vector strings;
+ if (!PyArg_ParseTuple(args, "isO", &zkhid, &path, &watcherfn))
+ return NULL;
+ pywatcher_t *pw = NULL;
+ if (watcherfn != Py_None)
+ pw = create_pywatcher( zkhid, watcherfn );
+ int err = zoo_wget_children( zhandles[zkhid], path,
+ watcherfn != Py_None ? watcher_dispatch : NULL,
+ pw, &strings );
+
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+
+ PyObject *ret = PyList_New(strings.count);
+ int i;
+ for (i=0;i<strings.count;++i)
+ {
+ PyObject *s = PyString_FromString(strings.data[i]);
+ PyList_SetItem(ret, i, s);
+ }
+ return ret;
+}
+
+static PyObject *pyzoo_set(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char *buffer;
+ int buflen;
+ int version;
+ if (!PyArg_ParseTuple(args, "iss#i", &zkhid, &path, &buffer, &buflen, &version))
+ return NULL;
+
+ assert(zkhid < num_zhandles);
+ int err = zoo_set(zhandles[zkhid], path, buffer, buflen, version);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+
+ return Py_BuildValue("i", err);
+}
+
+static PyObject *pyzoo_set2(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char *buffer;
+ int buflen;
+ int version;
+ if (!PyArg_ParseTuple(args, "iss#i", &zkhid, &path, &buffer, &buflen, &version))
+ return NULL;
+ struct Stat *stat = NULL;
+ int err = zoo_set2(zhandles[zkhid], path, buffer, buflen, version, stat);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+
+ return Py_BuildValue("O", build_stat(stat));
+}
+
+static PyObject *pyzoo_get(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ char buffer[512];
+ memset(buffer,0,sizeof(char)*512);
+ int buffer_len=512;
+ struct Stat stat;
+ PyObject *watcherfn = NULL;
+ pywatcher_t *pw = NULL;
+ if (!PyArg_ParseTuple(args, "isO", &zkhid, &path, &watcherfn))
+ return NULL;
+
+ if (watcherfn != Py_None)
+ pw = create_pywatcher( zkhid, watcherfn );
+ int err = zoo_wget(zhandles[zkhid], path,
+ watcherfn != Py_None ? watcher_dispatch : NULL,
+ pw, buffer,
+ &buffer_len, &stat);
+ PyObject *stat_dict = build_stat( &stat );
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+
+ return Py_BuildValue( "(s#,O)", buffer,buffer_len, stat_dict );
+}
+
+PyObject *pyzoo_get_acl(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ struct ACL_vector acl;
+ struct Stat stat;
+ if (!PyArg_ParseTuple(args, "is", &zkhid, &path))
+ return NULL;
+ int err = zoo_get_acl( zhandles[zkhid], path, &acl, &stat );
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+ PyObject *pystat = build_stat( &stat );
+ PyObject *acls = build_acls( &acl );
+ return Py_BuildValue( "(O,O)", pystat, acls );
+}
+
+PyObject *pyzoo_set_acl(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ char *path;
+ int version;
+ PyObject *pyacls;
+ struct ACL_vector acl;
+ if (!PyArg_ParseTuple(args, "isiO", &zkhid, &path, &version, &pyacls))
+ return NULL;
+
+ parse_acls(&acl, pyacls);
+ int err = zoo_set_acl(zhandles[zkhid], path, version, &acl );
+ free_acls(&acl);
+ if (err != ZOK)
+ {
+ PyErr_SetString( PyExc_IOError, zerror(err) );
+ return NULL;
+ }
+ return Py_BuildValue("i", err);;
+}
+
+///////////////////////////////////////////////////
+// session and context methods
+PyObject *pyzoo_close(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args, "i", &zkhid))
+ return NULL;
+ int ret = zookeeper_close(zhandles[zkhid]);
+ return Py_BuildValue("i", ret);
+}
+
+PyObject *pyzoo_client_id(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args, "i", &zkhid))
+ return NULL;
+ const clientid_t *cid = zoo_client_id(zhandles[zkhid]);
+ return Py_BuildValue("(L,s)", cid->client_id, cid->passwd);
+}
+
+PyObject *pyzoo_get_context(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args, "i", &zkhid))
+ return NULL;
+ PyObject *context = NULL;
+ context = (PyObject*)zoo_get_context(zhandles[zkhid]);
+ if (context) return context;
+ return Py_None;
+}
+
+PyObject *pyzoo_set_context(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ PyObject *context;
+ if (!PyArg_ParseTuple(args, "iO", &zkhid, &context))
+ return NULL;
+ zoo_set_context(zhandles[zkhid], (void*)context);
+ return Py_None;
+}
+///////////////////////////////////////////////////////
+// misc
+// static PyObject *generic_python_callback = NULL;
+// void generic_watcher(zhandle_t *zh, int type, int state, const char *path, void *watcherCtx)
+// {
+// if (generic_python_callback == NULL)
+// return;
+// }
+
+PyObject *pyzoo_set_watcher(PyObject *self, PyObject *args)
+{
+ /* int zkhid;
+ PyObject *watcherfn;
+ if (!PyArg_ParseTuple(args, "iO", &zkhid, &watcherfn))
+ return NULL;
+ watcher_fn cwatcher = zoo_set_watcher(
+ */
+ return NULL; // TODO
+}
+
+PyObject *pyzoo_state(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid))
+ return NULL;
+
+ int state = zoo_state(zhandles[zkhid]);
+ return Py_BuildValue("i",state);
+}
+
+
+// Synchronous calls return ZOK or throw an exception, but async calls get
+// an integer so should use this. This could perhaps use standardising.
+PyObject *pyzerror(PyObject *self, PyObject *args)
+{
+ int rc;
+ if (!PyArg_ParseTuple(args,"i", &rc))
+ return NULL;
+ return Py_BuildValue("s", zerror(rc));
+}
+
+PyObject *pyzoo_recv_timeout(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid))
+ return NULL;
+
+ int recv_timeout = zoo_recv_timeout(zhandles[zkhid]);
+ return Py_BuildValue("i",recv_timeout);
+}
+
+PyObject *pyis_unrecoverable(PyObject *self, PyObject *args)
+{
+ int zkhid;
+ if (!PyArg_ParseTuple(args,"i",&zkhid))
+ return NULL;
+ int ret = is_unrecoverable(zhandles[zkhid]);
+ return Py_BuildValue("i",ret); // TODO: make this a boolean
+}
+
+PyObject *pyzoo_set_debug_level(PyObject *self, PyObject *args)
+{
+ int loglevel;
+ if (!PyArg_ParseTuple(args, "i", &loglevel))
+ return NULL;
+ zoo_set_debug_level((ZooLogLevel)loglevel);
+ return Py_None;
+}
+static PyObject *log_stream = NULL;
+
+PyObject *pyzoo_set_log_stream(PyObject *self, PyObject *args)
+{
+ PyObject *pystream = NULL;
+ if (!PyArg_ParseTuple(args,"O",&pystream))
+ return NULL;
+ if (!PyFile_Check(pystream))
+ return NULL;
+ if (log_stream != NULL) {
+ Py_DECREF(log_stream);
+ }
+ log_stream = pystream;
+ Py_INCREF(log_stream);
+ zoo_set_log_stream(PyFile_AsFile(log_stream));
+
+ return Py_None;
+}
+
+PyObject *pyzoo_deterministic_conn_order(PyObject *self, PyObject *args)
+{
+ int yesOrNo;
+ if (!PyArg_ParseTuple(args, "i",&yesOrNo))
+ return NULL;
+ zoo_deterministic_conn_order( yesOrNo );
+ return Py_None;
+}
+
+///////////////////////////////////////////////////
+
+static PyMethodDef ZooKeeperMethods[] = {
+ {"init", pyzookeeper_init, METH_VARARGS },
+ {"create",pyzoo_create, METH_VARARGS },
+ {"delete",pyzoo_delete, METH_VARARGS },
+ {"get_children", pyzoo_get_children, METH_VARARGS },
+ {"set", pyzoo_set, METH_VARARGS },
+ {"set2", pyzoo_set2, METH_VARARGS },
+ {"get",pyzoo_get, METH_VARARGS },
+ {"exists",pyzoo_exists, METH_VARARGS },
+ {"get_acl", pyzoo_get_acl, METH_VARARGS },
+ {"set_acl", pyzoo_set_acl, METH_VARARGS },
+ {"close", pyzoo_close, METH_VARARGS },
+ {"client_id", pyzoo_client_id, METH_VARARGS },
+ {"get_context", pyzoo_get_context, METH_VARARGS },
+ {"set_context", pyzoo_set_context, METH_VARARGS },
+ // {"set_watcher", pyzoo_set_watcher, METH_VARARGS }, // Not yet implemented
+ {"state", pyzoo_state, METH_VARARGS },
+ {"recv_timeout",pyzoo_recv_timeout, METH_VARARGS },
+ {"is_unrecoverable",pyis_unrecoverable, METH_VARARGS },
+ {"set_debug_level",pyzoo_set_debug_level, METH_VARARGS },
+ {"set_log_stream",pyzoo_set_log_stream, METH_VARARGS },
+ {"deterministic_conn_order",pyzoo_deterministic_conn_order, METH_VARARGS },
+ {"acreate", pyzoo_acreate, METH_VARARGS },
+ {"adelete", pyzoo_adelete, METH_VARARGS },
+ {"aexists", pyzoo_aexists, METH_VARARGS },
+ {"aget", pyzoo_aget, METH_VARARGS },
+ {"aset", pyzoo_aset, METH_VARARGS },
+ {"aget_children", pyzoo_aget_children, METH_VARARGS },
+ {"async", pyzoo_async, METH_VARARGS },
+ {"aget_acl", pyzoo_aget_acl, METH_VARARGS },
+ {"aset_acl", pyzoo_aset_acl, METH_VARARGS },
+ {"zerror", pyzerror, METH_VARARGS },
+ {NULL, NULL}
+};
+
+#define ADD_INTCONSTANT(x) PyModule_AddIntConstant(module, #x, ZOO_##x)
+#define ADD_INTCONSTANTZ(x) PyModule_AddIntConstant(module, #x, Z##x)
+
+PyMODINIT_FUNC initzookeeper() {
+ PyEval_InitThreads();
+ PyObject *module = Py_InitModule("zookeeper", ZooKeeperMethods );
+ ADD_INTCONSTANT(PERM_READ);
+ ADD_INTCONSTANT(PERM_WRITE);
+ ADD_INTCONSTANT(PERM_CREATE);
+ ADD_INTCONSTANT(PERM_DELETE);
+ ADD_INTCONSTANT(PERM_ALL);
+ ADD_INTCONSTANT(PERM_ADMIN);
+
+ ADD_INTCONSTANT(EPHEMERAL);
+ ADD_INTCONSTANT(SEQUENCE);
+
+ ADD_INTCONSTANT(EXPIRED_SESSION_STATE);
+ ADD_INTCONSTANT(AUTH_FAILED_STATE);
+ ADD_INTCONSTANT(CONNECTING_STATE);
+ ADD_INTCONSTANT(ASSOCIATING_STATE);
+ ADD_INTCONSTANT(CONNECTED_STATE);
+
+ ADD_INTCONSTANT(CREATED_EVENT);
+ ADD_INTCONSTANT(DELETED_EVENT);
+ ADD_INTCONSTANT(CHANGED_EVENT);
+ ADD_INTCONSTANT(CHILD_EVENT);
+ ADD_INTCONSTANT(SESSION_EVENT);
+ ADD_INTCONSTANT(NOTWATCHING_EVENT);
+
+ ADD_INTCONSTANT(LOG_LEVEL_ERROR);
+ ADD_INTCONSTANT(LOG_LEVEL_WARN);
+ ADD_INTCONSTANT(LOG_LEVEL_INFO);
+ ADD_INTCONSTANT(LOG_LEVEL_DEBUG);
+
+ ADD_INTCONSTANTZ(SYSTEMERROR);
+ ADD_INTCONSTANTZ(RUNTIMEINCONSISTENCY);
+ ADD_INTCONSTANTZ(DATAINCONSISTENCY);
+ ADD_INTCONSTANTZ(CONNECTIONLOSS);
+ ADD_INTCONSTANTZ(MARSHALLINGERROR);
+ ADD_INTCONSTANTZ(UNIMPLEMENTED);
+ ADD_INTCONSTANTZ(OPERATIONTIMEOUT);
+ ADD_INTCONSTANTZ(BADARGUMENTS);
+ ADD_INTCONSTANTZ(INVALIDSTATE);
+
+ ADD_INTCONSTANTZ(OK);
+ ADD_INTCONSTANTZ(APIERROR);
+ ADD_INTCONSTANTZ(NONODE);
+ ADD_INTCONSTANTZ(NOAUTH);
+ ADD_INTCONSTANTZ(BADVERSION);
+ ADD_INTCONSTANTZ(NOCHILDRENFOREPHEMERALS);
+ ADD_INTCONSTANTZ(NODEEXISTS);
+ ADD_INTCONSTANTZ(NOTEMPTY);
+ ADD_INTCONSTANTZ(SESSIONEXPIRED);
+ ADD_INTCONSTANTZ(INVALIDCALLBACK);
+ ADD_INTCONSTANTZ(INVALIDACL);
+ ADD_INTCONSTANTZ(AUTHFAILED);
+ ADD_INTCONSTANTZ(CLOSING);
+ ADD_INTCONSTANTZ(NOTHING);
+}
+
+int main(int argc, char **argv)
+{
+ Py_SetProgramName(argv[0]);
+ Py_Initialize();
+ initzookeeper();
+ return 0;
+}
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/setup.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/setup.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/setup.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/setup.py Thu May 14 06:37:59 2009
@@ -0,0 +1,31 @@
+# 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.
+
+from distutils.core import setup, Extension
+
+zookeeper_basedir = "../../../"
+
+zookeepermodule = Extension("zookeeper",
+ sources=["src/c/zookeeper.c"],
+ include_dirs=[zookeeper_basedir + "/src/c/include",
+ zookeeper_basedir + "/src/c/generated"],
+ libraries=["zookeeper_mt"],
+ library_dirs=["/usr/local/lib"])
+
+setup( name="ZooKeeper",
+ version = "0.3",
+ description = "ZooKeeper Python bindings",
+ ext_modules=[zookeepermodule] )
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/zk.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/zk.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/zk.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/python/zk.py Thu May 14 06:37:59 2009
@@ -0,0 +1,76 @@
+# 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.
+
+import zookeeper, time, threading
+
+f = open("out.log","w")
+zookeeper.set_log_stream(f)
+
+connected = False
+conn_cv = threading.Condition( )
+
+def my_connection_watcher(handle,type,state,path):
+ global connected, conn_cv
+ print "Connected, handle is ", handle
+ conn_cv.acquire()
+ connected = True
+ conn_cv.notifyAll()
+ conn_cv.release()
+
+conn_cv.acquire()
+print "Connecting to localhost:2181 -- "
+handle = zookeeper.init("localhost:2181", my_connection_watcher, 10000, 0)
+while not connected:
+ conn_cv.wait()
+conn_cv.release()
+
+def my_getc_watch( handle, type, state, path ):
+ print "Watch fired -- "
+ print type, state, path
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"};
+
+try:
+ zookeeper.create(handle, "/zk-python", "data", [ZOO_OPEN_ACL_UNSAFE], 0)
+ zookeeper.get_children(handle, "/zk-python", my_getc_watch)
+ for i in xrange(5):
+ print "Creating sequence node ", i, " ", zookeeper.create(handle, "/zk-python/sequencenode", "data", [ZOO_OPEN_ACL_UNSAFE], zookeeper.SEQUENCE )
+except:
+ pass
+
+def pp_zk(handle,root, indent = 0):
+ """Pretty print a zookeeper tree, starting at root"""
+ def make_path(child):
+ if root == "/":
+ return "/" + child
+ return root + "/" + child
+ children = zookeeper.get_children(handle, root, None)
+ out = ""
+ for i in xrange(indent):
+ out += "\t"
+ out += "|---"+root + " :: " + zookeeper.get(handle, root, None)[0]
+ print out
+ for child in children:
+ pp_zk(handle,make_path(child),indent+1)
+
+print "ZNode tree -- "
+pp_zk(handle,"/")
+
+print "Getting ACL / Stat for /zk-python --"
+(stat, acl) = zookeeper.get_acl(handle, "/zk-python")
+print "Stat:: ", stat
+print "Acl:: ", acl
+
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/connection_test.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/connection_test.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/connection_test.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/connection_test.py Thu May 14 06:37:59 2009
@@ -0,0 +1,44 @@
+# 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.
+
+import unittest, threading
+
+import zookeeper, zktestbase
+
+class ConnectionTest(zktestbase.TestBase):
+ """Test whether we can make a connection"""
+ def setUp(self):
+ pass
+
+ def testconnection(self):
+ cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ cv.acquire()
+ self.connected = True
+ cv.notify()
+ cv.release()
+
+ cv.acquire()
+ ret = zookeeper.init( self.host, connection_watcher, 10000, 0)
+ self.assertEqual(ret, zookeeper.OK)
+ cv.wait(15.0)
+ cv.release()
+
+ self.assertEqual(self.connected, True, "Connection timed out to " + self.host)
+
+if __name__ == '__main__':
+ unittest.main()
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/create_test.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/create_test.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/create_test.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/create_test.py Thu May 14 06:37:59 2009
@@ -0,0 +1,67 @@
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+class SyncCreationTest(zktestbase.TestBase):
+ """Test whether we can create znodes"""
+ # to do: startup and teardown via scripts?
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.delete(self.handle, "/zk-python-createtest",-1)
+ except:
+ pass
+
+ def test_sync_create(self):
+ ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.create(self.handle, "/zk-python-createtest", "nodecontents", [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ self.assertEqual(ret, zookeeper.OK)
+
+class AsyncCreationTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.delete(self.handle, "/zk-python-acreatetest",-1)
+ except:
+ pass
+
+ def test_async_create(self):
+ self.cv = threading.Condition()
+ def callback(handle, rc, value):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.cv.notify()
+ self.cv.release()
+
+ ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+ self.assertEqual(self.connected, True, "Not connected!")
+ self.cv.acquire()
+
+ ret = zookeeper.acreate(self.handle, "/zk-python-acreatetest", "nodecontents",
+ [ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL,
+ callback )
+ self.assertEqual(ret, zookeeper.OK, "acreate failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "acreate timed out")
+
+
+if __name__ == '__main__':
+ unittest.main()
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/exists_test.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/exists_test.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/exists_test.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/exists_test.py Thu May 14 06:37:59 2009
@@ -0,0 +1,65 @@
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+class SyncExistsTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-existstest","existstest", [ZOO_OPEN_ACL_UNSAFE],zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_sync_exists(self):
+ self.assertEqual(self.connected, True)
+ ret = zookeeper.exists(self.handle, "/zk-python-existstest", None)
+ self.assertNotEqual(ret, None, "/zk-python-existstest does not exist (possibly means creation failure)")
+
+class AsyncExistsTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-aexiststest","existstest",[ZOO_OPEN_ACL_UNSAFE],zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_async_exists(self):
+ self.cv = threading.Condition()
+ def callback(handle, rc, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.cv.notify()
+ self.cv.release()
+ self.rc = rc
+
+ self.assertEqual(self.connected, True)
+
+ self.cv.acquire()
+ ret = zookeeper.aexists(self.handle, "/zk-python-aexiststest", None,
+ callback )
+ self.assertEqual(ret, zookeeper.OK)
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+
+ self.assertEqual(self.callback_flag, True, "aexists timed out")
+ self.assertEqual(self.rc, zookeeper.OK, "Return code not ok:" + zookeeper.zerror(self.rc))
+
+
+if __name__ == '__main__':
+ unittest.main()
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/get_set_test.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/get_set_test.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/get_set_test.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/get_set_test.py Thu May 14 06:37:59 2009
@@ -0,0 +1,87 @@
+# 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.
+
+import zookeeper, zktestbase, unittest, threading
+ZOO_OPEN_ACL_UNSAFE = {"perms":0x1f, "scheme":"world", "id" :"anyone"}
+
+class SyncGetSetTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-getsettest", "on",[ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_sync_getset(self):
+ self.assertEqual(self.connected, True, "Not connected!")
+ (data,stat) = zookeeper.get(self.handle, "/zk-python-getsettest", None)
+ self.assertEqual(data, "on", "Data is not 'on' as expected: " + data)
+ ret = zookeeper.set(self.handle, "/zk-python-getsettest", "off", stat["version"])
+ (data,stat) = zookeeper.get(self.handle, "/zk-python-getsettest", None)
+ self.assertEqual(data, "off", "Data is not 'off' as expected: " + data)
+
+class AsyncGetSetTest(zktestbase.TestBase):
+ def setUp( self ):
+ zktestbase.TestBase.setUp(self)
+ try:
+ zookeeper.create(self.handle, "/zk-python-agetsettest", "on",[ZOO_OPEN_ACL_UNSAFE], zookeeper.EPHEMERAL)
+ except:
+ pass
+
+ def test_async_getset(self):
+ self.cv = threading.Condition()
+ def get_callback(handle, rc, value, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.value = (value,stat)
+ self.cv.notify()
+ self.cv.release()
+
+ def set_callback(handle, rc, stat):
+ self.cv.acquire()
+ self.callback_flag = True
+ self.rc = rc
+ self.value = stat
+ self.cv.notify()
+ self.cv.release()
+
+ self.assertEqual(self.connected, True, "Not connected!")
+
+ self.cv.acquire()
+ self.callback_flag = False
+ ret = zookeeper.aset(self.handle, "/zk-python-agetsettest", "off", -1, set_callback)
+ self.assertEqual(ret, zookeeper.OK, "aset failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+ self.assertEqual(self.callback_flag, True, "aset timed out")
+
+ self.cv.acquire()
+ self.callback_flag = False
+ ret = zookeeper.aget(self.handle, "/zk-python-agetsettest", None, get_callback)
+ self.assertEqual(ret, zookeeper.OK, "aget failed")
+ while not self.callback_flag:
+ self.cv.wait(15)
+ self.cv.release()
+ self.assertEqual(self.callback_flag, True, "aget timed out")
+ self.assertEqual(self.value[0], "off", "Data is not 'off' as expected: " + self.value[0])
+
+
+
+
+if __name__ == '__main__':
+ unittest.main()
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/run_tests.sh
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/run_tests.sh?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/run_tests.sh (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/run_tests.sh Thu May 14 06:37:59 2009
@@ -0,0 +1,20 @@
+# 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.
+for test in `ls $1/*_test.py`;
+do
+ echo "Running $test"
+ python $test
+done
\ No newline at end of file
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zkServer.sh
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zkServer.sh?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zkServer.sh (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zkServer.sh Thu May 14 06:37:59 2009
@@ -0,0 +1,57 @@
+#!/bin/bash
+#
+# 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.
+
+if [ "x$1" == "x" ]
+then
+ echo "USAGE: $0 startClean|start|stop hostPorts"
+ exit 2
+fi
+
+if [ "x$1" == "xstartClean" ]
+then
+ if [ "x${base_dir}" == "x" ]
+ then
+ rm -rf /tmp/zkdata
+ else
+ rm -rf ${base_dir}/build/tmp
+ fi
+fi
+
+# Make sure nothing is left over from before
+#fuser -skn tcp 22182/tcp
+
+case $1 in
+start|startClean)
+ if [ "x${base_dir}" == "x" ]
+ then
+ mkdir -p /tmp/zkdata
+ java -cp ../../../zookeeper-dev.jar:../../../src/java/lib/log4j-1.2.15.jar:../../../conf org.apache.zookeeper.server.ZooKeeperServerMain 22182 /tmp/zkdata &> /tmp/zk.log &
+ else
+ mkdir -p ${base_dir}/build/tmp/zkdata
+ java -cp ${base_dir}/zookeeper-dev.jar:${base_dir}/src/java/lib/log4j-1.2.15.jar:${base_dir}/conf org.apache.zookeeper.server.ZooKeeperServerMain 22182 ${base_dir}/build/tmp/zkdata &> ${base_dir}/build/tmp/zk.log &
+ fi
+ sleep 5
+ ;;
+stop)
+ # Already killed above
+ ;;
+*)
+ echo "Unknown command " + $1
+ exit 2
+esac
+
Added: hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zktestbase.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zktestbase.py?rev=774656&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zktestbase.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/zkpython/src/test/zktestbase.py Thu May 14 06:37:59 2009
@@ -0,0 +1,49 @@
+# 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.
+
+import unittest, threading, zookeeper
+
+class TestBase(unittest.TestCase):
+ def __init__(self,methodName='runTest'):
+ unittest.TestCase.__init__(self,methodName)
+ self.host = "localhost:22182"
+ self.connected = False
+ self.handle = -1
+ f = open("build/test/logs/" + self.__class__.__name__ +".log","w")
+ zookeeper.set_log_stream(f)
+
+ def setUp(self):
+ self.callback_flag = False
+ self.cv = threading.Condition()
+ self.connected = False
+ def connection_watcher(handle, type, state, path):
+ self.cv.acquire()
+ self.connected = True
+ self.cv.notify()
+ self.cv.release()
+
+ self.cv.acquire()
+ self.handle = zookeeper.init( self.host, connection_watcher, 10000, 0)
+ self.cv.wait(15.0)
+ self.cv.release()
+
+ if not self.connected:
+ raise Exception("Couldn't connect to host -", self.host)
+
+ def tearDown(self):
+ if self.connected:
+ zookeeper.close(self.handle)
+
|