Repository: qpid-proton
Updated Branches:
  refs/heads/aconway-libuv-driver 464c166c2 -> 37c52232e (forced update)


http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/37c52232/proton-c/include/proton/connection_engine.h
----------------------------------------------------------------------
diff --git a/proton-c/include/proton/connection_engine.h 
b/proton-c/include/proton/connection_engine.h
index d9df77b..bae27d5 100644
--- a/proton-c/include/proton/connection_engine.h
+++ b/proton-c/include/proton/connection_engine.h
@@ -20,48 +20,31 @@
  * under the License.
  */
 
-///@file
-///
-/// **Experimental** The Connection Engine API wraps up the proton engine
-/// objects associated with a single connection: pn_connection_t, 
pn_transport_t
-/// and pn_collector_t. It provides a simple bytes-in/bytes-out interface for 
IO
-/// and generates pn_event_t events to be handled by the application.
-///
-/// The connection engine can be fed with raw AMQP bytes from any source, and 
it
-/// generates AMQP byte output to be written to any destination. You can use 
the
-/// engine to integrate proton AMQP with any IO library, or native IO on any
-/// platform.
-///
-/// The engine is not thread safe but each engine is independent. Separate
-/// engines can be used concurrently. For example a multi-threaded application
-/// can process connections in multiple threads, but serialize work for each
-/// connection to the corresponding engine.
-///
-/// The engine is designed to be thread and IO neutral so it can be integrated 
with
-/// single or multi-threaded code in reactive or proactive IO frameworks.
-///
-/// Summary of use:
-///
-/// - while !pn_connection_engine_finished()
-///   - Call pn_connection_engine_dispatch() to dispatch events until it 
returns NULL.
-///   - Read data from your source into pn_connection_engine_read_buffer()
-///   - Call pn_connection_engine_read_done() when complete.
-///   - Write data from pn_connection_engine_write_buffer() to your 
destination.
-///   - Call pn_connection_engine_write_done() to indicate how much was 
written.
-///
-/// Note on blocking: the _read/write_buffer and _read/write_done functions can
-/// all generate events that may cause the engine to finish. Before you wait 
for
-/// IO, always drain pn_connection_engine_dispatch() till it returns NULL and
-/// check pn_connection_engine_finished() in case there is nothing more to do..
-///
-/// Note on error handling: the pn_connection_engine_*() functions do not 
return
-/// an error code. If an error occurs it will be reported as a
-/// PN_TRANSPORT_ERROR event and pn_connection_engine_finished() will return
-/// true once all final events have been processed.
-///
-/// @defgroup connection_engine The Connection Engine
-/// @{
-///
+
+/**
+ * @file
+ * @defgroup connection_engine Connection Engine
+ * @ingroup engine
+ *
+ * **Experimental**: AMQP input bytes into proton @ref event "events", and
+ * application calls to the @ref engine API into AMQP output bytes.
+ *
+ * Each individual engine is not thread safe but separate engines can be used
+ * concurrently. A multi-threaded application must serialize activity for each
+ * connection but can process separate connections concurrently.
+ *
+ * Note on blocking: the _read/write_buffer and _read/write_done functions can
+ * generate events that may cause the engine to finish. Before you wait for IO,
+ * always drain pn_connection_engine_dispatch() till it returns NULL and check
+ * pn_connection_engine_finished() in case there is nothing more to do.
+ *
+ * Note on error handling: the pn_connection_engine_*() functions do not return
+ * an error code. If an error occurs it will be reported as a
+ * PN_TRANSPORT_ERROR event and pn_connection_engine_finished() will return
+ * true once all final events have been processed.
+ *
+ * @{
+ */
 
 #include <proton/condition.h>
 #include <proton/event.h>
@@ -72,11 +55,11 @@
 extern "C" {
 #endif
 
-/// A connection engine is a trio of pn_connection_t, pn_transport_t and 
pn_collector_t.
-/// Use the pn_connection_engine_*() functions to operate on it.
-/// It is a plain struct, not a proton object. Use pn_connection_engine_init 
to set up
-/// the initial objects and pn_connection_engine_final to release them.
-///
+/**
+ * connection_engine is a plain struct, not a proton object.  Use
+ * pn_connection_engine_init to initialize and pn_connection_engine_final 
before
+ * freeing.
+ */
 typedef struct pn_connection_engine_t {
     pn_connection_t* connection;
     pn_transport_t* transport;
@@ -84,98 +67,126 @@ typedef struct pn_connection_engine_t {
     pn_event_t* event;
 } pn_connection_engine_t;
 
-/// Initialize a pn_connection_engine_t struct with a new connection and
-/// transport.
-///
-/// Call pn_connection_engine_final to free resources when you are done.
-///
-///@return 0 on success, a proton error code on failure (@see error.h)
-///
+/**
+ * Initialize a pn_connection_engine_t struct, create a pn_connection_t and
+ * pn_transport_t. You can configure security properties on the connection,
+ * call connection_engine_start() to bind the transport before using the 
engine.
+ *
+ * @return 0 on success, a proton error code on failure (@see error.h)
+ */
 PN_EXTERN int pn_connection_engine_init(pn_connection_engine_t* engine);
 
-/// Free resources used by the engine, set the connection and transport 
pointers
-/// to NULL.
+/**
+ * Free resources used by the engine, set the connection and transport pointers
+ * to NULL.
+ */
 PN_EXTERN void pn_connection_engine_final(pn_connection_engine_t* engine);
 
-/// Get the engine's read buffer. Read data from your IO source to buf.start, 
up
-/// to a max of buf.size. Then call pn_connection_engine_read_done().
-///
-/// buf.size==0 means the engine cannot read presently, calling
-/// pn_connection_engine_dispatch() may create more buffer space.
-///
+/**
+ * Get the engine's read buffer. Read data from your IO source to buf.start, up
+ * to a max of buf.size. Then call pn_connection_engine_read_done().
+ *
+ * buf.size==0 means the engine cannot read presently, calling
+ * pn_connection_engine_dispatch() may create more buffer space.
+ */
 PN_EXTERN pn_rwbytes_t 
pn_connection_engine_read_buffer(pn_connection_engine_t*);
 
-/// Consume the first n bytes of data in pn_connection_engine_read_buffer() and
-/// update the buffer.
+/**
+ * Consume the first n bytes of data in pn_connection_engine_read_buffer() and
+ * update the buffer.
+ */
 PN_EXTERN void pn_connection_engine_read_done(pn_connection_engine_t*, size_t 
n);
 
-/// Close the read side of the transport when no more data is available.
-/// Note there may still be events for pn_connection_engine_dispatch() or data
-/// in pn_connection_engine_write_buffer()
+/**
+ * Close the read side of the transport when no more data is available.
+ * Note there may still be events for pn_connection_engine_dispatch() or data
+ * in pn_connection_engine_write_buffer()
+ */
 PN_EXTERN void pn_connection_engine_read_close(pn_connection_engine_t*);
 
-/// Get the engine's write buffer. Write data from buf.start to your IO 
destination,
-/// up to a max of buf.size. Then call pn_connection_engine_write_done().
-///
-/// buf.size==0 means the engine has nothing to write presently.  Calling
-/// pn_connection_engine_dispatch() may generate more data.
+/**
+ * Get the engine's write buffer. Write data from buf.start to your IO 
destination,
+ * up to a max of buf.size. Then call pn_connection_engine_write_done().
+
+ * buf.size==0 means the engine has nothing to write presently.  Calling
+ * pn_connection_engine_dispatch() may generate more data.
+ */
 PN_EXTERN pn_bytes_t 
pn_connection_engine_write_buffer(pn_connection_engine_t*);
 
-/// Call when the first n bytes of pn_connection_engine_write_buffer() have 
been
-/// written to IO and can be re-used for new data.  Updates the buffer.
+/**
+ * Call when the first n bytes of pn_connection_engine_write_buffer() have been
+ * written to IO and can be re-used for new data.  Updates the buffer.
+ */
 PN_EXTERN void pn_connection_engine_write_done(pn_connection_engine_t*, size_t 
n);
 
-/// Call when the write side of IO has closed and no more data can be written.
-/// Note that there may still be events for pn_connection_engine_dispatch() or
-/// data to read into pn_connection_engine_read_buffer().
+/**
+ * Call when the write side of IO has closed and no more data can be written.
+ * Note that there may still be events for pn_connection_engine_dispatch() or
+ * data to read into pn_connection_engine_read_buffer().
+ */
 PN_EXTERN void pn_connection_engine_write_close(pn_connection_engine_t*);
 
-/// Close both sides of the transport, equivalent to
-///     pn_connection_engine_read_close(); pn_connection_engine_write_close()
-///
-/// You must still call pn_connection_engine_dispatch() to process final
-/// events.
-///
-/// To provide transport error information to the handler, set it on
-///     pn_connection_engine_condition()
-/// *before* calling pn_connection_engine_disconnected(). This sets
-/// the error on the pn_transport_t object.
-///
-/// Note this does *not* modify the pn_connection_t, so you can distinguish
-/// between a connection close error sent by the remote peer (which will set 
the
-/// connection condition) and a transport error (which sets the transport
-/// condition.)
-///
+/**
+ * Close both sides of the transport, equivalent to
+ *
+ *     pn_connection_engine_read_close(); pn_connection_engine_write_close()
+ *
+ * To pass a transport error to the handler, set it on
+ * pn_connection_engine_condition() *before* calling
+ * pn_connection_engine_disconnected().  Note this is different from an AMQP
+ * close sent by the remote peer, which sets the connection condition.
+ *
+ * You must still call pn_connection_engine_dispatch() to process final
+ * events.
+ */
 PN_EXTERN void pn_connection_engine_disconnected(pn_connection_engine_t*);
 
-/// Get the next available event.
-/// Call in a loop until it returns NULL to dispatch all available events.
-/// Note this call may modify the read and write buffers.
-///
-/// @return Pointer to the next event, or NULL if there are none available.
-///
+/**
+ * Get the next available event.
+ * Call in a loop until it returns NULL to dispatch all available events.
+ * Note this call may modify the read and write buffers.
+ *
+ * @return Pointer to the next event, or NULL if there are none available.
+ *
+ */
 PN_EXTERN pn_event_t* pn_connection_engine_dispatch(pn_connection_engine_t*);
 
-/// Return true if the engine is finished - all data has been written, all
-/// events have been handled and the transport is closed.
+/**
+ * Return true if the engine is finished - all data has been written, all
+ * events have been handled and the transport is closed.
+ */
 PN_EXTERN bool pn_connection_engine_finished(pn_connection_engine_t*);
 
-/// Get the AMQP connection, owned by the pn_connection_engine_t.
+/**
+ * Return true if the engine transport is closed. There may still be
+ * outstanding events to process.
+
+ * Check if the transport has been closed internally (e.g. by an authentication
+ * failure) before blocking for IO.
+ */
+PN_EXTERN bool pn_connection_engine_closed(pn_connection_engine_t*);
+
+/**
+ * Get the AMQP connection, owned by the pn_connection_engine_t.
+ */
 PN_EXTERN pn_connection_t* 
pn_connection_engine_connection(pn_connection_engine_t*);
 
-/// Get the proton transport, owned by the pn_connection_engine_t.
+/**
+ * Get the proton transport, owned by the pn_connection_engine_t.
+ */
 PN_EXTERN pn_transport_t* 
pn_connection_engine_transport(pn_connection_engine_t*);
 
-/// Get the condition object for the engine's transport.
-///
-/// Note that IO errors should be set on this, the transport condition, not on
-/// the pn_connection_t condition. The connection's condition is for errors
-/// received via the AMQP protocol, the transport condition is for errors in 
the
-/// the IO layer such as a socket read or disconnect errors.
-///
+/**
+ * Get the condition object for the engine's transport.
+ *
+ * Note that IO errors should be set on this, the transport condition, not on
+ * the pn_connection_t condition. The connection's condition is for errors
+ * received via the AMQP protocol, the transport condition is for errors in the
+ * the IO layer such as a socket read or disconnect errors.
+ */
 PN_EXTERN pn_condition_t* 
pn_connection_engine_condition(pn_connection_engine_t*);
 
-///@}
+/** @}*/
 
 #ifdef __cplusplus
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/37c52232/proton-c/include/proton/driver.h
----------------------------------------------------------------------
diff --git a/proton-c/include/proton/driver.h b/proton-c/include/proton/driver.h
new file mode 100644
index 0000000..145b3c3
--- /dev/null
+++ b/proton-c/include/proton/driver.h
@@ -0,0 +1,282 @@
+#ifndef PROTON_DRIVER_H
+#define PROTON_DRIVER_H
+
+/*
+ * 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.
+ */
+
+/* FIXME aconway 2016-10-13: TOO:
+   - handle transport ticks
+   - support for scheduled wakeup (leave task queueing outside like conn 
wakeup)
+   - check when driver is "empty" - not monitoring anything. For clean 
shutdown.
+*/
+
+/*@file
+  @defgroup driver
+
+
+*/
+#include <proton/types.h>
+#include <proton/import_export.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+typedef struct pn_connection_engine_t pn_connection_engine_t;
+
+/**
+ * @defgroup driver The proton Driver API.
+ *
+ * **Experimental**: IO driver for a multi-threaded proton application
+ *
+ * The driver hides the details of the underlying IO platform, and handles IO
+ * read/write events.  Multiple threads call pn_driver_wait(), which returns
+ * a @ref pn_connection_engine_t when it has events to dispatch.
+ *
+ * Each connection can only be processed in a single thread at a time. The
+ * pn_driver_wait() threads dispatch all the available events, then call
+ * pn_driver_watch() to let the driver resume monitoring IO for the connection.
+ * Different connections can be processed concurrently.
+ *
+ * The driver allows you to:
+ *
+ * - create outgoing connections
+ * - listen for incoming connections
+ * - wake connections for application processing
+ *
+ * Waking a connection causes it to be returned by a  pn_driver_wait() call 
even
+ * if there are no IO events pending, so that you can safely do application
+ * processing that is not triggered by IO.
+ *
+ * ## Thread Safety
+ *
+ * Functions that take a pn_driver_t* parameter are thread safe.
+ *
+ * Unless noted otherwise, functions that take a pn_driver_connection_t* or
+ * pn_driver_listener_t* are not thread safe. They can be called sequentially
+ * after the connection/listener has been returned by pn_driver_wait() and 
until
+ * the connection/listener has been passed back to the driver by
+ * pn_driver_watch() or pn_driver_accept()
+ *
+ * ## Error handling
+ *
+ * Driver functions do not return an error code. Errors are indicated by
+ * PN_DRIVER_CONNECTION_FINISHED or PN_DRIVER_LISTENER_FINISHED events where 
the
+ * connection or listener carries the error information.
+ *
+ * ## Context information
+ *
+ * You can get the pn_driver_connection_t* associated with a pn_connection_t* 
via
+ * pn_driver_connection_get(). You can attach arbitrary additional data to the
+ * driver connection without extra allocations with this technique:
+
+ struct my_connection {
+ pn_driver_connection_t driver_conn;
+ // your additional data members here.
+ }
+ *
+ * Pass a pointer to the driver_conn member in a my_connection struct to
+ * pn_driver_connection_init() pn_driver_connect() and pn_driver_accept().  
When
+ * pn_driver_wait() returns a pn_driver_connection_t*, you can cast it to
+ * my_connection* to access your data.
+ *
+ * You should not use pn_connection_context() with driver connections as
+ * the driver may use it internally.
+
+ *
+ * ## Ease of use features
+ *
+ * This driver provides minimal features to hide the underlying IO platorm.
+ * Additional features can be layered on top, but are not built-in.
+ *
+ * For example: a feature to "inject" function objects to a connection can be
+ * implemented by using the pn_driver_wake() function and associating a
+ * queue of functions with the connection, this is left for higher layers
+ * because the best way to implement it will depend on the environment.
+ *
+ * @{
+ */
+
+/**
+ * The driver struct, initialize with pn_driver_init
+ */
+typedef struct pn_driver_t pn_driver_t;
+
+/**
+ * The driver connection struct, initialize with pn_driver_connection_init.
+ * Call pn_driver_engine() to get the contained pn_connection_engine_t.
+ */
+typedef struct pn_driver_connection_t pn_driver_connection_t;
+
+/**
+ * The driver listener struct, initialize with pn_driver_listener_init
+ */
+typedef struct pn_driver_listener_t pn_driver_listener_t;
+
+/**
+ * Type of event returned by pn_driver_wait()
+ */
+typedef enum {
+    PN_DRIVER_CONNECTION_READY,    /**< Connection ready for dispatch */
+    PN_DRIVER_CONNECTION_FINISHED, /**< Connection no longer active */
+    PN_DRIVER_LISTENER_READY,      /**< Listener ready for accept */
+    PN_DRIVER_LISTENER_FINISHED,   /**< Listener no longer active */
+    PN_DRIVER_INTERRUPT            /**< pn_driver_interrupt() called */
+} pn_driver_event_type_t;
+
+/**
+ * Event returned by pn_driver_wait()
+ */
+typedef struct pn_driver_event_t {
+    pn_driver_event_type_t type;
+    union {
+        pn_driver_connection_t *connection;
+        pn_driver_listener_t *listener;
+    };
+} pn_driver_event_t;
+
+/**
+ * Initialize a pn_driver_t struct
+ */
+void pn_driver_init(pn_driver_t *d);
+
+/**
+ * Finalize a driver struct, free all resources. Driver itself can be freed 
afterwards.
+ */
+void pn_driver_final(pn_driver_t *d);
+
+/**
+ * Wait for a driver event. Can be called in multiple threads concurrently.
+ * It is safe to use the connection/listener in the returned event until it is
+ * passed back to the driver via pn_driver_watch() or pn_driver_accept()
+ */
+pn_driver_event_t pn_driver_wait(pn_driver_t* d);
+
+/**
+ * Return PN_DRIVER_INTERRUPT in a single thread calling wait(). Thread safe.
+ */
+void pn_driver_interrupt(pn_driver_t* d);
+
+/**
+ * Initialize a driver connection struct
+ */
+void pn_driver_connection_init(pn_driver_t*, pn_driver_connection_t*);
+
+/**
+ * Finalize a driver connection after it has been returned by a 
PN_DRIVER_CONNECTION_FINISHED.
+ * Can be freed after.
+ */
+void pn_driver_connection_final(pn_driver_connection_t *c);
+
+/**
+ * Connect to host:service with an initialized pn_driver_connection_t (thread
+ * safe).  When there are events to dispatch, c it will be returned by
+ * pn_driver_wait() in a PN_DRIVER_CONNECTION_READY event. When c is finished
+ * and can be finalized and freed, it will be returned in a
+ * PN_DRIVER_CONNECTION_FINISHED event.
+ *
+ * @param c initialized driver connection struct
+ * @param host network host name
+ * @param service network service (aka port) name
+ * @param network can be NULL, placeholder for future multi-network drivers.
+ */
+void pn_driver_connect(pn_driver_connection_t* c, const char* host, const 
char* service, const char *network);
+
+/**
+ * Get the pn_connection_engine_t owned by a connection returned by 
pn_driver_wait().
+ */
+pn_connection_engine_t *pn_driver_engine(pn_driver_connection_t *c);
+
+/**
+ * Pass a connection that was previously returned by pn_driver_wait() back to 
the
+ * driver so it can monitor IO. It is not safe to use the connection until it 
is
+ * returned again by pn_driver_wait().
+ */
+void pn_driver_watch(pn_driver_connection_t *c);
+
+/**
+ * Cause a PN_DRIVER_CONNECTION_READY event for dc to be returned as soon as
+ * possible, even if there are no IO events (thread safe).
+ */
+void pn_driver_wake(pn_driver_connection_t *dc);
+
+/**
+ * Initialize a driver listener
+ */
+void pn_driver_listener_init(pn_driver_t*, pn_driver_listener_t *);
+
+/**
+ * Finalize a driver listener
+ */
+void pn_driver_listener_final(pn_driver_listener_t *);
+
+/**
+ * Listen on host:service with an initialized pn_driver_listener_t (thread 
safe).
+ * When there is an incoming connection pn_driver_wait() will return 
PN_DRIVER_LISTENER_READY,
+ * you should accept it with pn_driver_accept(). When the listener is closed 
it will
+ * be returned in a PN_DRIVER_LISTENER_FINISHED event.
+ *
+ * @param l initialized driver listener struct
+ * @param host network host name
+ * @param service network service (aka port) name
+ * @param network can be NULL, placeholder for future multi-network drivers.
+ * @param backlog number of incoming connections to buffer.
+ */
+void pn_driver_listen(pn_driver_listener_t* l, const char* host, const char* 
service, const char *network, int backlog);
+
+/**
+ * Accept a new connection on a listener returned in a PN_DRIVER_LISTENER_READY
+ * event by pn_driver_wait() It is not safe to use l or c until they are
+ * returned again by pn_driver_wait()
+ *
+ * @param l a ready listener, returned in a PN_DRIVER_LISTENER_READY event.
+ * @param c an initialized connection, not previously used for connect or 
accept.
+ */
+void pn_driver_accept(pn_driver_listener_t* l, pn_driver_connection_t* c);
+
+/**
+ * Close the listener (thread safe).
+ * Once closed it will be returned in a PN_DRIVER_LISTENER_FINISHED event.
+ */
+void pn_driver_listener_close(pn_driver_listener_t *l);
+
+/**
+ * Get the error (if any) on a listener returned in a 
PN_DRIVER_LISTENER_FINISHED event.
+ */
+const char* pn_driver_listener_error(pn_driver_listener_t*);
+
+/** Get the pn_driver_connection_t associated with a pn_connection_t if there 
is one */
+pn_driver_connection_t *pn_driver_connection_get(pn_connection_t*);
+
+/**
+ * @}
+ */
+
+#ifdef __cplusplus
+}
+#endif
+
+/* FIXME aconway 2016-10-20: Build flags to get consistent link/include of 
driver. Exports. */
+#ifdef PN_DRIVER_INCLUDE
+#include PN_DRIVER_INCLUDE
+#else
+#error "Define PN_DRIVER_INCLUDE as the driver implementation header file"
+#endif
+
+#endif // PROTON_DRIVER_IMPL_H

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/37c52232/proton-c/src/driver/driver.c
----------------------------------------------------------------------
diff --git a/proton-c/src/driver/driver.c b/proton-c/src/driver/driver.c
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/proton-c/src/driver/driver.c
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/37c52232/proton-c/src/engine/connection_engine.c
----------------------------------------------------------------------
diff --git a/proton-c/src/engine/connection_engine.c 
b/proton-c/src/engine/connection_engine.c
index 5d184a1..e1d1181 100644
--- a/proton-c/src/engine/connection_engine.c
+++ b/proton-c/src/engine/connection_engine.c
@@ -37,14 +37,17 @@ int pn_connection_engine_init(pn_connection_engine_t* e) {
 }
 
 void pn_connection_engine_final(pn_connection_engine_t* e) {
-    if (e->transport && e->connection) {
+    if (e->transport) {
         pn_transport_unbind(e->transport);
-        pn_decref(e->transport);
+        pn_transport_free(e->transport);
     }
-    if (e->collector)
+    if (e->collector) {
+        pn_collector_release(e->collector);
         pn_collector_free(e->collector); /* Break cycle with connection */
-    if (e->connection)
-        pn_decref(e->connection);
+    }
+    if (e->connection) {
+        pn_connection_free(e->connection);
+    }
     memset(e, 0, sizeof(*e));
 }
 
@@ -111,6 +114,10 @@ bool pn_connection_engine_finished(pn_connection_engine_t* 
e) {
     return pn_transport_closed(e->transport) && 
(pn_collector_peek(e->collector) == NULL);
 }
 
+bool pn_connection_engine_closed(pn_connection_engine_t* e) {
+    return pn_transport_closed(e->transport);
+}
+
 pn_connection_t* pn_connection_engine_connection(pn_connection_engine_t* e) {
     return e->connection;
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/37c52232/tools/cmake/Modules/FindLibuv.cmake
----------------------------------------------------------------------
diff --git a/tools/cmake/Modules/FindLibuv.cmake 
b/tools/cmake/Modules/FindLibuv.cmake
new file mode 100644
index 0000000..ae3ab70
--- /dev/null
+++ b/tools/cmake/Modules/FindLibuv.cmake
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+# Find libuv include dirs and libraries.
+#
+# Sets the following variables:
+#
+#   Libuv_FOUND            - True if headers and requested libraries were found
+#   Libuv_INCLUDE_DIRS     - Libuv include directories
+#   Libuv_LIBRARIES        - Link these to use libuv.
+#
+# This module reads hints about search locations from variables::
+#   LIBUV_ROOT             - Preferred installation prefix
+#   LIBUV_INCLUDEDIR       - Preferred include directory e.g. <prefix>/include
+#   LIBUV_LIBRARYDIR       - Preferred library directory e.g. <prefix>/lib
+
+find_library(Libuv_LIBRARIES Names uv libuv HINTS ${LIBUV_LIBRARYDIR} 
${LIBUV_ROOT})
+find_path(Libuv_INCLUDE_DIRS NAMES uv.h HINTS ${LIBUV_INCLUDEDIR} 
${LIBUV_ROOT} ${LIBUV_ROOT}/include ${CMAKE_INSTALL_PREFIX}/include PATHS 
/usr/include)
+
+include(FindPackageHandleStandardArgs)
+find_package_handle_standard_args(Libuv DEFAULT_MSG Libuv_LIBRARIES 
Libuv_INCLUDE_DIRS)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@qpid.apache.org
For additional commands, e-mail: commits-h...@qpid.apache.org

Reply via email to