You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ac...@apache.org on 2018/11/19 22:53:36 UTC

[6/6] qpid-proton git commit: PROTON-1887: [c] Convert C tests to use Catch2 harness.

PROTON-1887: [c] Convert C tests to use Catch2 harness.

Convert c/tests from home-baked test macros to the Catch2 test harness.
We are using version 1.x until we can move on from supporting C++03.
https://github.com/catchorg/Catch2/tree/Catch1.x

Added files:

test/include/
- catch.hpp - the Catch2 version 1.x framework in a single header.
- catch_extra.hpp - added Catch::Matcher support for C-string comparisons

c/tests/
- pn_test.hpp/.cpp - tools for testing the core library
  - auto_free<> - scoped freeing of pn_xxx_t*
  - matchers to check pn_condition/error_t with useful output
  - driver class to pump a pn_connection_driver_t and check events
    - trivial in-memory copy transport, no networking needed.
- pn_test_proactor.hpp/cpp - tools for testing the proactor library
  - proactor to pump a pn_proactor and check events.
  - same handler as driver above.

Renamed tests files: c/tests/*.c => c/tests/*_test.cpp

Tests are now linked into 3 executables:

- c-core-test - tests for libqpid-proton-core API
- c-extra-test - tests for the "extra" deprecated API in libqpid-proton
- c-proactor-test - tests for the proactor API in libqpid-proton-proactor

Individual tests or subsets can be run via command line arguments, run with
"-h" for a summary, or see https://github.com/catchorg/Catch2/blob/Catch1.x/docs/command-line.md

NOTE: Catch2 allows multiple "tags" to be associated with each test to easily
define different test sets. We're not using this feature yet, we could use it to
identify a fast self-test set, a long-running stress test set etc.


Project: http://git-wip-us.apache.org/repos/asf/qpid-proton/repo
Commit: http://git-wip-us.apache.org/repos/asf/qpid-proton/commit/0bdba37d
Tree: http://git-wip-us.apache.org/repos/asf/qpid-proton/tree/0bdba37d
Diff: http://git-wip-us.apache.org/repos/asf/qpid-proton/diff/0bdba37d

Branch: refs/heads/master
Commit: 0bdba37da9fbce7db4db106c21c1393ef6f1b433
Parents: f53c768
Author: Alan Conway <ac...@redhat.com>
Authored: Wed Nov 14 17:12:51 2018 -0500
Committer: Alan Conway <ac...@redhat.com>
Committed: Mon Nov 19 17:46:12 2018 -0500

----------------------------------------------------------------------
 c/src/proactor/proactor-internal.h |    12 +-
 c/tests/CMakeLists.txt             |    78 +-
 c/tests/condition.c                |    56 -
 c/tests/condition_test.cpp         |    50 +
 c/tests/connection_driver.c        |   591 --
 c/tests/connection_driver_test.cpp |   567 ++
 c/tests/data.c                     |   104 -
 c/tests/data_test.cpp              |    92 +
 c/tests/engine.c                   |   369 -
 c/tests/engine_test.cpp            |   318 +
 c/tests/event.c                    |   109 -
 c/tests/event_test.cpp             |   105 +
 c/tests/message.c                  |    96 -
 c/tests/message_test.cpp           |    84 +
 c/tests/object.c                   |  1115 ---
 c/tests/object_test.cpp            |   969 +++
 c/tests/parse-url.c                |   127 -
 c/tests/pn_test.cpp                |   217 +
 c/tests/pn_test.hpp                |   193 +
 c/tests/pn_test_proactor.cpp       |   159 +
 c/tests/pn_test_proactor.hpp       |   120 +
 c/tests/pn_test_test.cpp           |    67 +
 c/tests/proactor.c                 |  1107 ---
 c/tests/proactor_test.cpp          |   831 +++
 c/tests/reactor.c                  |   578 --
 c/tests/refcount.c                 |   393 -
 c/tests/refcount_test.cpp          |   371 +
 c/tests/ssl.c                      |    71 -
 c/tests/ssl_test.cpp               |    60 +
 c/tests/test_handler.h             |   184 -
 c/tests/test_main.cpp              |    46 +
 c/tests/test_tools.h               |   262 -
 c/tests/url_test.cpp               |   137 +
 cpp/CMakeLists.txt                 |    20 +-
 cpp/src/cpp-test.cpp               |    21 +
 cpp/src/url_test.cpp               |   124 +-
 tests/include/catch.hpp            | 11685 ++++++++++++++++++++++++++++++
 tests/include/catch_extra.hpp      |   107 +
 38 files changed, 16325 insertions(+), 5270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/src/proactor/proactor-internal.h
----------------------------------------------------------------------
diff --git a/c/src/proactor/proactor-internal.h b/c/src/proactor/proactor-internal.h
index 67c0bf6..cc5f2cb 100644
--- a/c/src/proactor/proactor-internal.h
+++ b/c/src/proactor/proactor-internal.h
@@ -20,9 +20,13 @@
  * under the License.
  */
 
-#include <proton/type_compat.h>
-#include <proton/import_export.h>
 #include <proton/condition.h>
+#include <proton/import_export.h>
+#include <proton/type_compat.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
 
 /* NOTE PNP_EXTERN is for use by proton-internal tests  */
 
@@ -48,4 +52,8 @@ extern const char *PNI_IO_CONDITION;
 void pni_proactor_set_cond(
   pn_condition_t *cond, const char *what, const char *host, const char *port, const char *msg);
 
+#ifdef __cplusplus
+}
+#endif
+
 #endif  /*!PROACTOR_PROACTOR_INTERNAL_H*/

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/c/tests/CMakeLists.txt b/c/tests/CMakeLists.txt
index a4bfb67..f6957d4 100644
--- a/c/tests/CMakeLists.txt
+++ b/c/tests/CMakeLists.txt
@@ -17,51 +17,61 @@
 # under the License.
 #
 
-add_definitions(${COMPILE_WARNING_FLAGS} ${COMPILE_PLATFORM_FLAGS})
-
 configure_file(test_config.h.in test_config.h)
-include_directories(${CMAKE_CURRENT_BINARY_DIR})
+include_directories(${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_SOURCE_DIR}/tests/include)
+if (WIN32)
+  set(test_env ${PN_ENV_SCRIPT} -- "PATH=$<TARGET_FILE_DIR:qpid-proton-core>")
+else()
+  set(platform_test_src ssl_test.cpp)
+endif()
 
-set(test_env ${PN_ENV_SCRIPT} -- "PATH=$<TARGET_FILE_DIR:qpid-proton-core>")
+# NOTE: C library tests are written in C++ using the Catch2 framework.
+# The tests are more concise than with a native C framework and we can
+# use the same framework for C++ tests.
+# See https://github.com/catchorg/Catch2/blob/Catch1.x/docs/tutorial.md for more.
+# We will use the Catch2 1.x branch for as long as we must support C++03
+#
+# ${CMAKE_SOURCE_DIR}/include/tests/catch_extra.cpp has some general extensions to catch.
+# ./pn_test.hpp contains proton-C specific code to help with testing.
+#
+# Tests for each library are combined in a single executable.
 
-# Add test without any qpid-proton lib: allows choice of
-# qpid-proton-core or qpid-proton
-macro (pn_add_c_test_nolib test)
-  add_executable (${test} ${ARGN})
-  target_link_libraries (${test} ${PLATFORM_LIBS})
-  if (BUILD_WITH_CXX)
-    set_source_files_properties (${ARGN} PROPERTIES LANGUAGE CXX)
-  endif (BUILD_WITH_CXX)
-  add_test(NAME ${test} COMMAND ${test_env} ${TEST_EXE_PREFIX_CMD} $<TARGET_FILE:${test}>)
-endmacro (pn_add_c_test_nolib)
+macro(add_c_test exe)
+  add_executable(${exe} test_main.cpp pn_test.cpp ${ARGN})
+  set_target_properties(${exe} PROPERTIES
+    COMPILE_FLAGS "${CMAKE_CXX_FLAGS} ${CXX_WARNING_FLAGS}")
+  add_test(NAME ${exe} COMMAND ${test_env} ${TEST_EXE_PREFIX_CMD} $<TARGET_FILE:${exe}>)
+endmacro()
 
-# Add test with qpid-proton-core linked
-macro (pn_add_c_test test)
-  pn_add_c_test_nolib (${test} ${ARGN})
-  target_link_libraries (${test} qpid-proton-core ${PLATFORM_LIBS})
-endmacro (pn_add_c_test)
+## Tests that depend only on qpid-proton-core
+add_c_test(c-core-test
+  pn_test_test.cpp
+  object_test.cpp
+  condition_test.cpp
+  connection_driver_test.cpp
+  data_test.cpp
+  engine_test.cpp
+  refcount_test.cpp
+  ${platform_test_src})
 
-pn_add_c_test (c-object-tests object.c)
-pn_add_c_test (c-message-tests message.c)
-pn_add_c_test (c-engine-tests engine.c)
-pn_add_c_test (c-refcount-tests refcount.c)
-pn_add_c_test (c-event-tests event.c)
-pn_add_c_test (c-data-tests data.c)
-pn_add_c_test (c-condition-tests condition.c)
-pn_add_c_test (c-connection-driver-tests connection_driver.c)
-pn_add_c_test (c-ssl-tests ssl.c)
+target_link_libraries(c-core-test qpid-proton-core ${PLATFORM_LIBS})
 
-pn_add_c_test_nolib (c-parse-url-tests parse-url.c)
-target_link_libraries (c-parse-url-tests qpid-proton)
+## Tests for the deprecated "extra" part of the qpid-proton library.
+add_c_test(c-extra-test url_test.cpp)
+target_link_libraries(c-extra-test qpid-proton ${PLATFORM_LIBS})
 
 if(HAS_PROACTOR)
-  pn_add_c_test (c-proactor-tests proactor.c)
-  target_link_libraries (c-proactor-tests qpid-proton-proactor)
+  # Tests for qpid-proton-proactor
+  add_c_test(c-proactor-test pn_test_proactor.cpp proactor_test.cpp)
+  target_link_libraries(c-proactor-test qpid-proton-core qpid-proton-proactor ${PLATFORM_LIBS})
 
-  # TODO Enable by default when races and xcode problems are cleared up
+  # Thread race test.
+  #
+  # TODO aconway 2018-11-14: enable by default when races and xcode
+  # problems are cleared up
   option(THREADERCISER "Run the threaderciser concurrency tests" OFF)
   if (THREADERCISER)
-    pn_add_c_test(c-threaderciser threaderciser.c)
+    add_c_test(c-threaderciser threaderciser.c)
     target_link_libraries (c-threaderciser qpid-proton-proactor)
     find_library(Pthread_LIBRARY pthread)
     if (Pthread_LIBRARY)

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/condition.c
----------------------------------------------------------------------
diff --git a/c/tests/condition.c b/c/tests/condition.c
deleted file mode 100644
index 5ecc263..0000000
--- a/c/tests/condition.c
+++ /dev/null
@@ -1,56 +0,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.
- */
-
-#include <proton/condition.h>
-#include <proton/connection.h>
-#include <stdio.h>
-#include <string.h>
-
-static int fail = 0;
-
-#define TEST_ASSERT(B)                                  \
-    if(!(B)) {                                          \
-        ++fail;                                         \
-        printf("%s:%d %s\n", __FILE__, __LINE__ , #B); \
-    }
-
-int main(int argc, char **argv) {
-    pn_connection_t *c = pn_connection();
-    pn_condition_t *cond = pn_connection_condition(c);
-
-    // Verify empty
-    TEST_ASSERT(!pn_condition_is_set(cond));
-    TEST_ASSERT(!pn_condition_get_name(cond));
-    TEST_ASSERT(!pn_condition_get_description(cond));
-
-    // Format a condition
-    pn_condition_format(cond, "foo", "hello %d", 42);
-    TEST_ASSERT(pn_condition_is_set(cond));
-    TEST_ASSERT(strcmp("foo", pn_condition_get_name(cond)) == 0);
-    TEST_ASSERT(strcmp("hello 42", pn_condition_get_description(cond)) == 0);
-
-    // Clear and verify empty
-    pn_condition_clear(cond);
-    TEST_ASSERT(!pn_condition_is_set(cond));
-    TEST_ASSERT(!pn_condition_get_name(cond));
-    TEST_ASSERT(!pn_condition_get_description(cond));
-
-    pn_connection_free(c);
-    return fail;
-}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/condition_test.cpp
----------------------------------------------------------------------
diff --git a/c/tests/condition_test.cpp b/c/tests/condition_test.cpp
new file mode 100644
index 0000000..89e7648
--- /dev/null
+++ b/c/tests/condition_test.cpp
@@ -0,0 +1,50 @@
+/*
+ * 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 "./pn_test.hpp"
+
+#include <proton/condition.h>
+#include <proton/connection.h>
+
+using Catch::Matchers::Equals;
+using namespace pn_test;
+
+TEST_CASE("condition") {
+  auto_free<pn_connection_t, pn_connection_free> c(pn_connection());
+  REQUIRE(c);
+  pn_condition_t *cond = pn_connection_condition(c);
+  REQUIRE(cond);
+
+  // Verify empty
+  CHECK(!pn_condition_is_set(cond));
+  CHECK(!pn_condition_get_name(cond));
+  CHECK(!pn_condition_get_description(cond));
+
+  // Format a condition
+  pn_condition_format(cond, "foo", "hello %d", 42);
+  CHECK(pn_condition_is_set(cond));
+  CHECK_THAT("foo", Equals(pn_condition_get_name(cond)));
+  CHECK_THAT("hello 42", Equals(pn_condition_get_description(cond)));
+
+  // Clear and verify empty
+  pn_condition_clear(cond);
+  CHECK(!pn_condition_is_set(cond));
+  CHECK(!pn_condition_get_name(cond));
+  CHECK(!pn_condition_get_description(cond));
+}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/connection_driver.c
----------------------------------------------------------------------
diff --git a/c/tests/connection_driver.c b/c/tests/connection_driver.c
deleted file mode 100644
index c22c2d9..0000000
--- a/c/tests/connection_driver.c
+++ /dev/null
@@ -1,591 +0,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.
- */
-
-
-#include "test_handler.h"
-#include <proton/codec.h>
-#include <proton/connection_driver.h>
-#include <proton/connection.h>
-#include <proton/delivery.h>
-#include <proton/message.h>
-#include <proton/session.h>
-#include <proton/link.h>
-
-/* Handler that replies to REMOTE_OPEN, stores the opened object on the handler */
-static pn_event_type_t open_handler(test_handler_t *th, pn_event_t *e) {
-  switch (pn_event_type(e)) {
-   case PN_CONNECTION_REMOTE_OPEN:
-    th->connection = pn_event_connection(e);
-    pn_connection_open(th->connection);
-    break;
-   case PN_SESSION_REMOTE_OPEN:
-    th->session =  pn_event_session(e);
-    pn_session_open(th->session);
-    break;
-   case PN_LINK_REMOTE_OPEN:
-    th->link = pn_event_link(e);
-    pn_link_open(th->link);
-    break;
-   default:
-    break;
-  }
-  return PN_EVENT_NONE;
-}
-
-/* Like open_handler but also reply to REMOTE_CLOSE */
-static pn_event_type_t open_close_handler(test_handler_t *th, pn_event_t *e) {
-  switch (pn_event_type(e)) {
-   case PN_CONNECTION_REMOTE_CLOSE:
-    pn_connection_open(pn_event_connection(e));
-    break;
-   case PN_SESSION_REMOTE_CLOSE:
-    pn_session_open(pn_event_session(e));
-    break;
-   case PN_LINK_REMOTE_CLOSE:
-    pn_link_close(pn_event_link(e));
-    break;
-   default:
-    return open_handler(th, e);
-  }
-  return PN_EVENT_NONE;
-}
-
-/* Handler that returns control on PN_DELIVERY and stores the delivery on the handler */
-static pn_event_type_t delivery_handler(test_handler_t *th, pn_event_t *e) {
-  switch (pn_event_type(e)) {
-   case PN_DELIVERY: {
-     th->delivery = pn_event_delivery(e);
-    return PN_DELIVERY;
-   }
-   default:
-    return open_handler(th, e);
-  }
-}
-
-/* Blow-by-blow event verification of a single message transfer */
-static void test_message_transfer(test_t *t) {
-  test_connection_driver_t client, server;
-  test_connection_driver_init(&client, t, open_handler, NULL);
-  test_connection_driver_init(&server, t, delivery_handler, NULL);
-  pn_transport_set_server(server.driver.transport);
-
-  pn_connection_open(client.driver.connection);
-  pn_session_t *ssn = pn_session(client.driver.connection);
-  pn_session_open(ssn);
-  pn_link_t *snd = pn_sender(ssn, "x");
-  pn_link_open(snd);
-  test_connection_drivers_run(&client, &server);
-
-  TEST_HANDLER_EXPECT(
-    &client.handler,
-    PN_CONNECTION_INIT, PN_CONNECTION_LOCAL_OPEN,
-    PN_SESSION_INIT, PN_SESSION_LOCAL_OPEN,
-    PN_LINK_INIT, PN_LINK_LOCAL_OPEN,
-    PN_CONNECTION_BOUND, PN_CONNECTION_REMOTE_OPEN, PN_SESSION_REMOTE_OPEN, PN_LINK_REMOTE_OPEN,
-    0);
-
-  TEST_HANDLER_EXPECT(
-    &server.handler,
-    PN_CONNECTION_INIT, PN_CONNECTION_BOUND, PN_CONNECTION_REMOTE_OPEN,
-    PN_SESSION_INIT, PN_SESSION_REMOTE_OPEN,
-    PN_LINK_INIT, PN_LINK_REMOTE_OPEN,
-    PN_CONNECTION_LOCAL_OPEN, PN_TRANSPORT,
-    PN_SESSION_LOCAL_OPEN, PN_TRANSPORT,
-    PN_LINK_LOCAL_OPEN, PN_TRANSPORT,
-    0);
-
-  pn_link_t *rcv = server.handler.link;
-  TEST_CHECK(t, rcv);
-  TEST_CHECK(t, pn_link_is_receiver(rcv));
-  pn_link_flow(rcv, 1);
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT(&client.handler, PN_LINK_FLOW, 0);
-
-  /* Encode and send a message */
-  pn_message_t *m = pn_message();
-  pn_data_put_string(pn_message_body(m), pn_bytes(4, "abc")); /* Include trailing NULL */
-  pn_delivery(snd, PN_BYTES_LITERAL(x));
-  pn_message_send(m, snd, NULL);
-  pn_message_free(m);
-
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT(&server.handler, PN_TRANSPORT, PN_DELIVERY, 0);
-
-  /* Receive and decode the message */
-  pn_delivery_t *dlv = server.handler.delivery;
-  TEST_ASSERT(dlv);
-  pn_message_t *m2 = pn_message();
-  pn_rwbytes_t buf2 = { 0 };
-  message_decode(m2, dlv, &buf2);
-  pn_data_t *body = pn_message_body(m2);
-  pn_data_rewind(body);
-  TEST_CHECK(t, pn_data_next(body));
-  TEST_CHECK(t, PN_STRING == pn_data_type(body));
-  TEST_CHECK(t, 4 == pn_data_get_string(pn_message_body(m2)).size);
-  TEST_STR_EQUAL(t, "abc", pn_data_get_string(pn_message_body(m2)).start);
-  pn_message_free(m2);
-
-  free(buf2.start);
-  test_connection_driver_destroy(&client);
-  test_connection_driver_destroy(&server);
-}
-
-/* Handler that opens a connection and sender link */
-pn_event_type_t send_client_handler(test_handler_t *th, pn_event_t *e) {
-  switch (pn_event_type(e)) {
-   case PN_CONNECTION_LOCAL_OPEN: {
-    pn_connection_open(pn_event_connection(e));
-    pn_session_t *ssn = pn_session(pn_event_connection(e));
-    pn_session_open(ssn);
-    pn_link_t *snd = pn_sender(ssn, "x");
-    pn_link_open(snd);
-    break;
-   }
-   case PN_LINK_REMOTE_OPEN: {
-    th->link = pn_event_link(e);
-    return PN_LINK_REMOTE_OPEN;
-   }
-   default:
-    break;
-  }
-  return PN_EVENT_NONE;
-}
-
-/* Send a message in pieces, ensure each can be received before the next is sent */
-static void test_message_stream(test_t *t) {
-  /* Set up the link, give credit, start the delivery */
-  test_connection_driver_t client, server;
-  test_connection_driver_init(&client, t, send_client_handler, NULL);
-  test_connection_driver_init(&server, t, delivery_handler, NULL);
-  pn_transport_set_server(server.driver.transport);
-
-  pn_connection_open(client.driver.connection);
-  test_connection_drivers_run(&client, &server);
-  pn_link_t *rcv = server.handler.link;
-  pn_link_t *snd = client.handler.link;
-  pn_link_flow(rcv, 1);
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT_LAST(&client.handler, PN_LINK_FLOW);
-  TEST_HANDLER_EXPECT_LAST(&server.handler, PN_TRANSPORT);
-
-  /* Encode a large (not very) message to send in chunks */
-  pn_message_t *m = pn_message();
-  char body[1024] = { 0 };
-  pn_data_put_binary(pn_message_body(m), pn_bytes(sizeof(body), body));
-  pn_rwbytes_t buf = { 0 };
-  ssize_t size = message_encode(m, &buf);
-
-  /* Send and receive the message in chunks */
-  static const ssize_t CHUNK = 100;
-  pn_delivery(snd, PN_BYTES_LITERAL(x));
-  pn_rwbytes_t buf2 = { 0 };
-  ssize_t received = 0;
-  for (ssize_t i = 0; i < size; i += CHUNK) {
-    /* Send a chunk */
-    ssize_t c = (i+CHUNK < size) ? CHUNK : size - i;
-    TEST_CHECK(t, c == pn_link_send(snd, buf.start + i, c));
-    test_connection_drivers_run(&client, &server);
-    TEST_HANDLER_EXPECT_LAST(&server.handler, PN_DELIVERY);
-    /* Receive a chunk */
-    pn_delivery_t *dlv = server.handler.delivery;
-    pn_link_t *l = pn_delivery_link(dlv);
-    ssize_t dsize = pn_delivery_pending(dlv);
-    rwbytes_ensure(&buf2, received+dsize);
-    TEST_ASSERT(dsize == pn_link_recv(l, buf2.start + received, dsize));
-    received += dsize;
-  }
-  TEST_CHECK(t, pn_link_advance(snd));
-  TEST_CHECK(t, received == size);
-  TEST_CHECK(t, !memcmp(buf.start, buf2.start, size));
-
-  pn_message_free(m);
-  free(buf.start);
-  free(buf2.start);
-  test_connection_driver_destroy(&client);
-  test_connection_driver_destroy(&server);
-}
-
-// Test aborting a delivery
-static void test_message_abort(test_t *t) {
-  /* Set up the link, give credit, start the delivery */
-  test_connection_driver_t client, server;
-  test_connection_driver_init(&client, t, send_client_handler, NULL);
-  test_connection_driver_init(&server, t, delivery_handler, NULL);
-  pn_transport_set_server(server.driver.transport);
-  pn_connection_open(client.driver.connection);
-
-  test_connection_drivers_run(&client, &server);
-  pn_link_t *rcv = server.handler.link;
-  pn_link_t *snd = client.handler.link;
-  char data[100] = {0};          /* Dummy data to send. */
-  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
-
-  /* Send 2 frames with data */
-  pn_link_flow(rcv, 1);
-  TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-  test_connection_drivers_run(&client, &server);
-  TEST_INT_EQUAL(t, 1, pn_link_credit(snd));
-  pn_delivery_t *sd = pn_delivery(snd, PN_BYTES_LITERAL(1)); /* Sender delivery */
-  for (size_t i = 0; i < 2; ++i) {
-    TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-    test_connection_drivers_run(&client, &server);
-    TEST_HANDLER_EXPECT_LAST(&server.handler, PN_DELIVERY);
-    pn_delivery_t *rd = server.handler.delivery;
-    TEST_CHECK(t, !pn_delivery_aborted(rd));
-    TEST_CHECK(t, pn_delivery_partial(rd));
-    TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-    TEST_INT_EQUAL(t, sizeof(data), pn_delivery_pending(rd));
-    TEST_INT_EQUAL(t, sizeof(rbuf), pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
-    TEST_INT_EQUAL(t, 0, pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
-    TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-  }
-  TEST_INT_EQUAL(t, 1, pn_link_credit(snd));
-  /* Abort the delivery */
-  pn_delivery_abort(sd);
-  TEST_INT_EQUAL(t, 0, pn_link_credit(snd));
-  TEST_CHECK(t, pn_link_current(snd) != sd); /* Settled */
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT_LAST(&server.handler, PN_DELIVERY);
-  TEST_INT_EQUAL(t, 0, pn_link_credit(snd));
-
-  /* Receive the aborted=true frame, should be empty */
-  pn_delivery_t *rd = server.handler.delivery;
-  TEST_CHECK(t, pn_delivery_aborted(rd));
-  TEST_CHECK(t, !pn_delivery_partial(rd)); /* Aborted deliveries are never partial */
-  TEST_CHECK(t, pn_delivery_settled(rd)); /* Aborted deliveries are remote settled */
-  TEST_INT_EQUAL(t, 1, pn_delivery_pending(rd));
-  TEST_INT_EQUAL(t, PN_ABORTED, pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
-  pn_delivery_settle(rd);       /* Must be settled locally to free it */
-
-  TEST_INT_EQUAL(t, 0, pn_link_credit(snd));
-  TEST_INT_EQUAL(t, 0, pn_link_credit(rcv));
-
-  /* Abort a delivery before any data has been framed, should be dropped. */
-  pn_link_flow(rcv, 1);
-  TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-  test_connection_drivers_run(&client, &server);
-  test_handler_clear(&client.handler, 0);
-  test_handler_clear(&server.handler, 0);
-
-  sd = pn_delivery(snd, PN_BYTES_LITERAL(x));
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-  pn_delivery_abort(sd);
-  TEST_CHECK(t, pn_link_current(snd) != sd); /* Settled, possibly freed */
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT(&server.handler, 0); /* Expect no delivery at the server */
-  /* Client gets transport/flow after abort to ensure other messages are sent */
-  TEST_HANDLER_EXPECT(&client.handler, PN_TRANSPORT, PN_LINK_FLOW, 0);
-  /* Aborted delivery consumes no credit */
-  TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-  TEST_INT_EQUAL(t, 1, pn_link_credit(snd));
-
-  test_connection_driver_destroy(&client);
-  test_connection_driver_destroy(&server);
-}
-
-
-int send_receive_message(test_t *t, const char* tag,
-                         test_connection_driver_t *src, test_connection_driver_t *dst)
-{
-  int errors = t->errors;
-  char data[100] = {0};          /* Dummy data to send. */
-  strncpy(data, tag, sizeof(data)-1);
-  data[99] = 0; /* Ensure terminated as we strcmp this later*/
-
-  if (!TEST_CHECK(t, pn_link_credit(src->handler.link))) return 1;
-
-  pn_delivery_t *sd = pn_delivery(src->handler.link, pn_dtag(tag, strlen(tag)));
-  dst->handler.delivery = NULL;
-  TEST_CHECK(t, pn_delivery_current(sd));
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(src->handler.link, data, sizeof(data)));
-  pn_delivery_settle(sd);
-  test_connection_drivers_run(src, dst);
-  pn_delivery_t *rd = dst->handler.delivery;
-  dst->handler.delivery = NULL;
-  if (!TEST_CHECK(t, rd)) return 1;
-
-  TEST_CHECK(t, pn_delivery_current(rd));
-  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
-  TEST_INT_EQUAL(t, sizeof(rbuf), pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
-  TEST_STR_EQUAL(t, data, rbuf);
-  pn_delivery_settle(rd);
-  return t->errors > errors;
-}
-
-#define SEND_RECEIVE_MESSAGE(T, TAG, SRC, DST)                  \
-  TEST_INT_EQUAL(T, 0, send_receive_message(T, TAG, SRC, DST))
-
-// Test mixing aborted and good deliveries, make sure credit is correct.
-static void test_message_abort_mixed(test_t *t) {
-  /* Set up the link, give credit, start the delivery */
-  test_connection_driver_t client, server;
-  test_connection_driver_init(&client, t, send_client_handler, NULL);
-  test_connection_driver_init(&server, t, delivery_handler, NULL);
-  pn_transport_set_server(server.driver.transport);
-  pn_connection_open(client.driver.connection);
-
-  test_connection_drivers_run(&client, &server);
-  pn_link_t *rcv = server.handler.link;
-  pn_link_t *snd = client.handler.link;
-  char data[100] = {0};          /* Dummy data to send. */
-  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
-
-  /* We will send 3 good messages, interleaved with aborted ones */
-  pn_link_flow(rcv, 5);
-  test_connection_drivers_run(&client, &server);
-  SEND_RECEIVE_MESSAGE(t, "one", &client, &server);
-  TEST_INT_EQUAL(t, 4, pn_link_credit(snd));
-  TEST_INT_EQUAL(t, 4, pn_link_credit(rcv));
-  pn_delivery_t *sd, *rd;
-
-  /* Send a frame, then an abort */
-  sd = pn_delivery(snd, PN_BYTES_LITERAL("x1"));
-  server.handler.delivery = NULL;
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-  TEST_INT_EQUAL(t, 4, pn_link_credit(snd)); /* Nothing sent yet */
-  test_connection_drivers_run(&client, &server);
-  rd = server.handler.delivery;
-  if (!TEST_CHECK(t, rd)) goto cleanup;
-  TEST_INT_EQUAL(t, sizeof(rbuf), pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
-
-  pn_delivery_abort(sd);
-  test_connection_drivers_run(&client, &server);
-  TEST_CHECK(t, pn_delivery_aborted(rd));
-  pn_delivery_settle(rd);
-  /* Abort after sending data consumes credit */
-  TEST_INT_EQUAL(t, 3, pn_link_credit(snd));
-  TEST_INT_EQUAL(t, 3, pn_link_credit(rcv));
-
-  SEND_RECEIVE_MESSAGE(t, "two", &client, &server);
-  TEST_INT_EQUAL(t, 2, pn_link_credit(snd));
-  TEST_INT_EQUAL(t, 2, pn_link_credit(rcv));
-
-  /* Abort a delivery before any data has been framed, should be dropped. */
-  test_handler_clear(&server.handler, 0);
-  sd = pn_delivery(snd, PN_BYTES_LITERAL(4));
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-  pn_delivery_abort(sd);
-  TEST_CHECK(t, pn_link_current(snd) != sd); /* Advanced */
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT(&server.handler, PN_TRANSPORT, 0);
-  /* Aborting wit no frames sent should leave credit untouched */
-  TEST_INT_EQUAL(t, 2, pn_link_credit(snd));
-  TEST_INT_EQUAL(t, 2, pn_link_credit(rcv));
-
-  SEND_RECEIVE_MESSAGE(t, "three", &client, &server);
-  TEST_INT_EQUAL(t, 1, pn_link_credit(rcv));
-  TEST_INT_EQUAL(t, 1, pn_link_credit(snd));
-
- cleanup:
-  test_connection_driver_destroy(&client);
-  test_connection_driver_destroy(&server);
-}
-
-/* Set capacity and max frame, send a single message */
-static void set_capacity_and_max_frame(
-  size_t capacity, size_t max_frame,
-  test_connection_driver_t *client, test_connection_driver_t *server,
-  const char* data)
-{
-  pn_transport_set_max_frame(client->driver.transport, max_frame);
-  pn_connection_open(client->driver.connection);
-  pn_session_t *ssn = pn_session(client->driver.connection);
-  pn_session_set_incoming_capacity(ssn, capacity);
-  pn_session_open(ssn);
-  pn_link_t *snd = pn_sender(ssn, "x");
-  pn_link_open(snd);
-  test_connection_drivers_run(client, server);
-  pn_link_flow(server->handler.link, 1);
-  test_connection_drivers_run(client, server);
-  if (pn_transport_closed(client->driver.transport) ||
-      pn_transport_closed(server->driver.transport))
-    return;
-  /* Send a message */
-  pn_message_t *m = pn_message();
-  pn_message_set_address(m, data);
-  pn_delivery(snd, PN_BYTES_LITERAL(x));
-  pn_message_send(m, snd, NULL);
-  pn_message_free(m);
-  test_connection_drivers_run(client, server);
-}
-
-/* Test different settings for max-frame, outgoing-window, incoming-capacity */
-static void test_session_flow_control(test_t *t) {
-  test_connection_driver_t client, server;
-  test_connection_drivers_init(t, &client, open_handler, &server, delivery_handler);
-  pn_message_t *m = pn_message();
-  pn_rwbytes_t buf= {0};
-
-  /* Capacity equal to frame size OK */
-  set_capacity_and_max_frame(1234, 1234, &client, &server, "foo");
-  pn_delivery_t *dlv = server.handler.delivery;
-  if (TEST_CHECK(t, dlv)) {
-    message_decode(m, dlv, &buf);
-    TEST_STR_EQUAL(t, "foo", pn_message_get_address(m));
-  }
-
-  /* Capacity bigger than frame size OK */
-  set_capacity_and_max_frame(12345, 1234, &client, &server, "foo");
-  dlv = server.handler.delivery;
-  if (TEST_CHECK(t, dlv)) {
-    message_decode(m, dlv, &buf);
-    TEST_STR_EQUAL(t, "foo", pn_message_get_address(m));
-  }
-
-  /* Capacity smaller than frame size is an error */
-  set_capacity_and_max_frame(1234, 12345, &client, &server, "foo");
-  TEST_COND_NAME(t, "amqp:internal-error", pn_transport_condition(client.driver.transport));
-  TEST_COND_DESC(t, "session capacity 1234 is less than frame size 12345", pn_transport_condition(client.driver.transport));
-
-  pn_message_free(m);
-  free(buf.start);
-  test_connection_drivers_destroy(&client, &server);
-}
-
-/* Regression test for https://issues.apache.org/jira/browse/PROTON-1832.
-   Make sure we error on attempt to re-attach an already-attached link name.
-   No crash or memory error.
-*/
-static void test_duplicate_link_server(test_t *t) {
-  test_connection_driver_t client, server;
-  test_connection_drivers_init(t, &client, open_close_handler, &server, open_close_handler);
-  pn_connection_open(client.driver.connection);
-  pn_session_t *ssn = pn_session(client.driver.connection);
-  pn_session_open(ssn);
-
-  /* Set up link "x" */
-  pn_link_t *x = pn_sender(ssn, "xxx");
-  pn_link_open(x);
-  test_connection_drivers_run(&client, &server);
-  test_handler_clear(&client.handler, 0);
-  test_handler_clear(&server.handler, 0);
-  /* Free (but don't close) the link and open a new one to generate the invalid double-attach */
-  pn_link_free(x);
-  pn_link_open(pn_sender(ssn, "xxx"));
-  test_connection_drivers_run(&client, &server);
-
-  TEST_COND_NAME(t, "amqp:invalid-field", pn_transport_condition(server.driver.transport));
-  TEST_COND_DESC(t, "xxx", pn_transport_condition(server.driver.transport));
-
-  TEST_COND_NAME(t, "amqp:invalid-field", pn_connection_remote_condition(client.driver.connection));
-  TEST_COND_DESC(t, "xxx", pn_connection_remote_condition(client.driver.connection));
-
-  /* Freeing the link at this point is allowed but caused a crash in transport_unbind with the bug */
-  pn_link_free(server.handler.link);
-  test_connection_drivers_destroy(&client, &server);
-}
-
-/* Reproducer test for https://issues.apache.org/jira/browse/PROTON-1832.
-   Make sure the client does not generate an illegal "attach; attach; detach" sequence
-   from a legal "pn_link_open(); pn_link_close(); pn_link_open()" sequence
-*/
-static void test_duplicate_link_client(test_t *t) {
-  /* This test will fail till PROTON-1832 is fully fixed */
-  t->inverted = true;
-  /* Set up the initial link */
-  test_connection_driver_t client, server;
-  test_connection_drivers_init(t, &client, open_close_handler, &server, open_close_handler);
-  pn_connection_open(client.driver.connection);
-  pn_session_t *ssn = pn_session(client.driver.connection);
-  pn_session_open(ssn);
-  pn_link_t *x = pn_sender(ssn, "x");
-  pn_link_open(x);
-  test_connection_drivers_run(&client, &server);
-  test_handler_clear(&client.handler, 0);
-  test_handler_clear(&server.handler, 0);
-
-  /* Close the link and open a new link with same name in the same batch of events. */
-  pn_link_close(x);
-  pn_link_open(pn_sender(ssn, "x"));
-  test_connection_drivers_run(&client, &server);
-
-  TEST_HANDLER_EXPECT(&server.handler,
-                      PN_LINK_REMOTE_CLOSE, PN_LINK_LOCAL_CLOSE, PN_TRANSPORT,
-                      PN_LINK_INIT, PN_LINK_REMOTE_OPEN, PN_LINK_LOCAL_OPEN, PN_TRANSPORT,
-                      0);
-  TEST_COND_EMPTY(t, pn_transport_condition(server.driver.transport));
-
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT(&client.handler,
-                      PN_LINK_LOCAL_CLOSE, PN_TRANSPORT, PN_LINK_REMOTE_CLOSE,
-                      PN_LINK_INIT, PN_LINK_LOCAL_OPEN, PN_TRANSPORT, PN_LINK_REMOTE_OPEN,
-                      0);
-  TEST_COND_EMPTY(t, pn_connection_remote_condition(client.driver.connection));
-  test_connection_drivers_destroy(&client, &server);
-}
-
-/* Settling an incomplete delivery should not cause an error */
-static void test_settle_incomplete_receiver(test_t *t) {
-  /* Inverted: this test will fail till PROTON-1914 is fixed */
-  t->inverted = true;
-
-  /* Set up the link, give credit, start the delivery */
-  test_connection_driver_t client, server;
-  test_connection_driver_init(&client, t, send_client_handler, NULL);
-  test_connection_driver_init(&server, t, delivery_handler, NULL);
-  pn_transport_set_server(server.driver.transport);
-  pn_connection_open(client.driver.connection);
-
-  test_connection_drivers_run(&client, &server);
-  pn_link_t *rcv = server.handler.link;
-  pn_link_t *snd = client.handler.link;
-  char data[100] = {0};          /* Dummy data to send. */
-  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
-  pn_link_flow(rcv, 1);
-  pn_delivery(snd, PN_BYTES_LITERAL(1)); /* Prepare to send */
-  test_connection_drivers_run(&client, &server);
-
-  /* Send/receive a frame */
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT_LAST(&server.handler, PN_DELIVERY);
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_recv(rcv, rbuf, sizeof(data)));
-  test_connection_drivers_run(&client, &server);
-
-  /* Settle the receiver's delivery */
-  pn_delivery_settle(pn_link_current(rcv));
-  test_connection_drivers_run(&client, &server);
-  TEST_COND_EMPTY(t, pn_connection_remote_condition(client.driver.connection));
-  TEST_COND_EMPTY(t, pn_connection_condition(server.driver.connection));
-
-  /* Send/receive a frame, should not cause error */
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_send(snd, data, sizeof(data)));
-  test_connection_drivers_run(&client, &server);
-  TEST_HANDLER_EXPECT_LAST(&server.handler, PN_DELIVERY);
-  TEST_INT_EQUAL(t, sizeof(data), pn_link_recv(rcv, rbuf, sizeof(data)));
-  test_connection_drivers_run(&client, &server);
-  TEST_COND_EMPTY(t, pn_connection_remote_condition(client.driver.connection));
-  TEST_COND_EMPTY(t, pn_connection_condition(server.driver.connection));
-
-  test_connection_driver_destroy(&client);
-  test_connection_driver_destroy(&server);
-}
-
-int main(int argc, char **argv) {
-  int failed = 0;
-  RUN_ARGV_TEST(failed, t, test_message_transfer(&t));
-  RUN_ARGV_TEST(failed, t, test_message_stream(&t));
-  RUN_ARGV_TEST(failed, t, test_message_abort(&t));
-  RUN_ARGV_TEST(failed, t, test_message_abort_mixed(&t));
-  RUN_ARGV_TEST(failed, t, test_session_flow_control(&t));
-  RUN_ARGV_TEST(failed, t, test_duplicate_link_server(&t));
-  RUN_ARGV_TEST(failed, t, test_duplicate_link_client(&t));
-  RUN_ARGV_TEST(failed, t, test_settle_incomplete_receiver(&t));
-  return failed;
-}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/connection_driver_test.cpp
----------------------------------------------------------------------
diff --git a/c/tests/connection_driver_test.cpp b/c/tests/connection_driver_test.cpp
new file mode 100644
index 0000000..175a77d
--- /dev/null
+++ b/c/tests/connection_driver_test.cpp
@@ -0,0 +1,567 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ h * 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 "./pn_test.hpp"
+
+#include <proton/codec.h>
+#include <proton/connection.h>
+#include <proton/connection_driver.h>
+#include <proton/delivery.h>
+#include <proton/link.h>
+#include <proton/message.h>
+#include <proton/session.h>
+#include <proton/transport.h>
+
+#include <string.h>
+
+using Catch::Matchers::EndsWith;
+using Catch::Matchers::Equals;
+using namespace pn_test;
+
+namespace {
+
+/* Handler that replies to REMOTE_OPEN, stores the opened object on the handler
+ */
+struct open_handler : pn_test::handler {
+  bool handle(pn_event_t *e) CATCH_OVERRIDE {
+    switch (pn_event_type(e)) {
+    case PN_CONNECTION_REMOTE_OPEN:
+      connection = pn_event_connection(e);
+      pn_connection_open(connection);
+      break;
+    case PN_SESSION_REMOTE_OPEN:
+      session = pn_event_session(e);
+      pn_session_open(session);
+      break;
+    case PN_LINK_REMOTE_OPEN:
+      link = pn_event_link(e);
+      pn_link_open(link);
+      break;
+    default:
+      break;
+    }
+    return false;
+  }
+};
+
+/* Like open_handler but also reply to REMOTE_CLOSE */
+struct open_close_handler : public open_handler {
+  bool handle(pn_event_t *e) CATCH_OVERRIDE {
+    switch (pn_event_type(e)) {
+    case PN_CONNECTION_REMOTE_CLOSE:
+      pn_connection_open(pn_event_connection(e));
+      break;
+    case PN_SESSION_REMOTE_CLOSE:
+      pn_session_open(pn_event_session(e));
+      break;
+    case PN_LINK_REMOTE_CLOSE:
+      pn_link_close(pn_event_link(e));
+      break;
+    default:
+      return open_handler::handle(e);
+    }
+    return false;
+  }
+};
+
+/* open_handler that returns control on PN_DELIVERY and stores the delivery */
+struct delivery_handler : public open_handler {
+  bool handle(pn_event_t *e) {
+    switch (pn_event_type(e)) {
+    case PN_DELIVERY: {
+      delivery = pn_event_delivery(e);
+      return true;
+    }
+    default:
+      return open_handler::handle(e);
+    }
+  }
+};
+
+} // namespace
+
+/* Blow-by-blow event verification of a single message transfer */
+TEST_CASE("driver_message_transfer") {
+  open_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  pn_connection_open(d.client.connection);
+  pn_session_t *ssn = pn_session(d.client.connection);
+  pn_session_open(ssn);
+  pn_link_t *snd = pn_sender(ssn, "x");
+  pn_link_open(snd);
+  d.run();
+
+  CHECK_THAT(ETYPES(PN_CONNECTION_INIT, PN_CONNECTION_LOCAL_OPEN,
+                    PN_SESSION_INIT, PN_SESSION_LOCAL_OPEN, PN_LINK_INIT,
+                    PN_LINK_LOCAL_OPEN, PN_CONNECTION_BOUND,
+                    PN_CONNECTION_REMOTE_OPEN, PN_SESSION_REMOTE_OPEN,
+                    PN_LINK_REMOTE_OPEN),
+             Equals(client.log_clear()));
+
+  CHECK_THAT(ETYPES(PN_CONNECTION_INIT, PN_CONNECTION_BOUND,
+                    PN_CONNECTION_REMOTE_OPEN, PN_SESSION_INIT,
+                    PN_SESSION_REMOTE_OPEN, PN_LINK_INIT, PN_LINK_REMOTE_OPEN,
+                    PN_CONNECTION_LOCAL_OPEN, PN_TRANSPORT,
+                    PN_SESSION_LOCAL_OPEN, PN_TRANSPORT, PN_LINK_LOCAL_OPEN,
+                    PN_TRANSPORT),
+             Equals(server.log_clear()));
+
+  pn_link_t *rcv = server.link;
+  REQUIRE(rcv);
+  REQUIRE(pn_link_is_receiver(rcv));
+  pn_link_flow(rcv, 1);
+  d.run();
+  CHECK_THAT(ETYPES(PN_LINK_FLOW), Equals(client.log_clear()));
+
+  /* Encode and send a message */
+  auto_free<pn_message_t, pn_message_free> m(pn_message());
+  pn_data_put_string(pn_message_body(m),
+                     pn_bytes("abc")); /* Include trailing NULL */
+  pn_delivery(snd, pn_bytes("x"));
+  pn_message_send(m, snd, NULL);
+
+  d.run();
+  CHECK_THAT(ETYPES(PN_TRANSPORT, PN_DELIVERY), Equals(server.log_clear()));
+
+  /* Receive and decode the message */
+  pn_delivery_t *dlv = server.delivery;
+  REQUIRE(dlv);
+  auto_free<pn_message_t, pn_message_free> m2(pn_message());
+  pn_rwbytes_t buf2 = {0};
+  message_decode(m2, dlv, &buf2);
+  pn_data_t *body = pn_message_body(m2);
+  pn_data_rewind(body);
+  CHECK(pn_data_next(body));
+  CHECK(PN_STRING == pn_data_type(body));
+  CHECK(3 == pn_data_get_string(pn_message_body(m2)).size);
+  CHECK_THAT("abc", Equals(pn_data_get_string(pn_message_body(m2)).start));
+
+  free(buf2.start);
+}
+
+namespace {
+/* Handler that opens a connection and sender link */
+struct send_client_handler : public pn_test::handler {
+  bool handle(pn_event_t *e) {
+    switch (pn_event_type(e)) {
+    case PN_CONNECTION_LOCAL_OPEN: {
+      pn_connection_open(pn_event_connection(e));
+      pn_session_t *ssn = pn_session(pn_event_connection(e));
+      pn_session_open(ssn);
+      pn_link_t *snd = pn_sender(ssn, "x");
+      pn_link_open(snd);
+      break;
+    }
+    case PN_LINK_REMOTE_OPEN: {
+      link = pn_event_link(e);
+      return true;
+    }
+    default:
+      break;
+    }
+    return false;
+  }
+};
+} // namespace
+
+/* Send a message in pieces, ensure each can be received before the next is sent
+ */
+TEST_CASE("driver_message_stream") {
+  send_client_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  d.run();
+  pn_link_t *rcv = server.link;
+  pn_link_t *snd = client.link;
+  pn_link_flow(rcv, 1);
+  d.run();
+  CHECK(PN_LINK_FLOW == client.log_last());
+  CHECK(PN_TRANSPORT == server.log_last());
+
+  /* Encode a large (not very) message to send in chunks */
+  auto_free<pn_message_t, pn_message_free> m(pn_message());
+  char body[1024] = {0};
+  pn_data_put_binary(pn_message_body(m), pn_bytes(sizeof(body), body));
+  pn_rwbytes_t buf = {0};
+  ssize_t size = pn_message_encode2(m, &buf);
+
+  /* Send and receive the message in chunks */
+  static const ssize_t CHUNK = 100;
+  pn_delivery(snd, pn_bytes("x"));
+  pn_rwbytes_t buf2 = {0};
+  ssize_t received = 0;
+  for (ssize_t i = 0; i < size; i += CHUNK) {
+    /* Send a chunk */
+    ssize_t c = (i + CHUNK < size) ? CHUNK : size - i;
+    CHECK(c == pn_link_send(snd, buf.start + i, c));
+    d.run();
+    CHECK_THAT(ETYPES(PN_DELIVERY), Equals(server.log_clear()));
+    /* Receive a chunk */
+    pn_delivery_t *dlv = server.delivery;
+    pn_link_t *l = pn_delivery_link(dlv);
+    ssize_t dsize = pn_delivery_pending(dlv);
+    rwbytes_ensure(&buf2, received + dsize);
+    REQUIRE(dsize == pn_link_recv(l, buf2.start + received, dsize));
+    received += dsize;
+  }
+  CHECK(pn_link_advance(snd));
+  CHECK(received == size);
+  CHECK(!memcmp(buf.start, buf2.start, size));
+
+  free(buf.start);
+  free(buf2.start);
+}
+
+// Test aborting a delivery
+TEST_CASE("driver_message_abort") {
+  send_client_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  d.run();
+  pn_link_t *rcv = server.link;
+  pn_link_t *snd = client.link;
+  char data[100] = {0};          /* Dummy data to send. */
+  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
+
+  /* Send 2 frames with data */
+  pn_link_flow(rcv, 1);
+  CHECK(1 == pn_link_credit(rcv));
+  d.run();
+  CHECK(1 == pn_link_credit(snd));
+  pn_delivery_t *sd = pn_delivery(snd, pn_bytes("1")); /* Sender delivery */
+  for (size_t i = 0; i < 2; ++i) {
+    CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+    d.run();
+    CHECK(server.log_last() == PN_DELIVERY);
+    pn_delivery_t *rd = server.delivery;
+    CHECK(!pn_delivery_aborted(rd));
+    CHECK(pn_delivery_partial(rd));
+    CHECK(1 == pn_link_credit(rcv));
+    CHECK(sizeof(data) == pn_delivery_pending(rd));
+    CHECK(sizeof(rbuf) ==
+          pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
+    CHECK(0 == pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
+    CHECK(1 == pn_link_credit(rcv));
+  }
+  CHECK(1 == pn_link_credit(snd));
+  /* Abort the delivery */
+  pn_delivery_abort(sd);
+  CHECK(0 == pn_link_credit(snd));
+  CHECK(pn_link_current(snd) != sd); /* Settled */
+  d.run();
+  CHECK(PN_DELIVERY == server.log_last());
+  CHECK(0 == pn_link_credit(snd));
+
+  /* Receive the aborted=true frame, should be empty */
+  pn_delivery_t *rd = server.delivery;
+  CHECK(pn_delivery_aborted(rd));
+  CHECK(!pn_delivery_partial(rd)); /* Aborted deliveries are never partial */
+  CHECK(pn_delivery_settled(rd));  /* Aborted deliveries are remote settled */
+  CHECK(1 == pn_delivery_pending(rd));
+  CHECK(PN_ABORTED == pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
+  pn_delivery_settle(rd); /* Must be settled locally to free it */
+
+  CHECK(0 == pn_link_credit(snd));
+  CHECK(0 == pn_link_credit(rcv));
+
+  /* Abort a delivery before any data has been framed, should be dropped. */
+  pn_link_flow(rcv, 1);
+  CHECK(1 == pn_link_credit(rcv));
+  d.run();
+  client.log_clear();
+  server.log_clear();
+
+  sd = pn_delivery(snd, pn_bytes("x"));
+  CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+  pn_delivery_abort(sd);
+  CHECK(pn_link_current(snd) != sd); /* Settled, possibly freed */
+  CHECK_FALSE(d.run());
+  CHECK_THAT(server.log_clear(), Equals(etypes()));
+  /* Client gets transport/flow after abort to ensure other messages are sent */
+  CHECK_THAT(ETYPES(PN_TRANSPORT, PN_LINK_FLOW), Equals(client.log_clear()));
+  /* Aborted delivery consumes no credit */
+  CHECK(1 == pn_link_credit(rcv));
+  CHECK(1 == pn_link_credit(snd));
+}
+
+void send_receive_message(const std::string &tag, pn_test::driver_pair &d) {
+  pn_link_t *l = d.client.handler.link;
+  CHECKED_IF(pn_link_credit(l) > 0) {
+    pn_delivery_t *sd = pn_delivery(l, pn_dtag(tag.data(), tag.size()));
+    d.server.handler.delivery = NULL;
+    CHECK(pn_delivery_current(sd));
+    CHECK(tag.size() == pn_link_send(l, tag.data(), tag.size()));
+    pn_delivery_settle(sd);
+    d.run();
+    pn_delivery_t *rd = d.server.handler.delivery;
+    d.server.handler.delivery = NULL;
+    CHECKED_IF(rd) {
+      CHECK(pn_delivery_current(rd));
+      std::string rbuf(tag.size() * 2, 'x');
+      CHECK(tag.size() ==
+            pn_link_recv(pn_delivery_link(rd), &rbuf[0], rbuf.size()));
+      rbuf.resize(tag.size());
+      CHECK(tag == rbuf);
+    }
+    pn_delivery_settle(rd);
+  }
+}
+
+#define SEND_RECEIVE_MESSAGE(TAG, DP)                                          \
+  do {                                                                         \
+    INFO("in send_receive_message: " << TAG);                                  \
+    send_receive_message(TAG, DP);                                             \
+  } while (0)
+
+// Test mixing aborted and good deliveries, make sure credit is correct.
+TEST_CASE("driver_message_abort_mixed") {
+  send_client_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  d.run();
+  pn_link_t *rcv = server.link;
+  pn_link_t *snd = client.link;
+  char data[100] = {0};          /* Dummy data to send. */
+  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
+
+  /* We will send 3 good messages, interleaved with aborted ones */
+  pn_link_flow(rcv, 5);
+  d.run();
+  SEND_RECEIVE_MESSAGE("one", d);
+  CHECK(4 == pn_link_credit(snd));
+  CHECK(4 == pn_link_credit(rcv));
+  pn_delivery_t *sd, *rd;
+
+  /* Send a frame, then an abort */
+  sd = pn_delivery(snd, pn_bytes("x1"));
+  server.delivery = NULL;
+  CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+  CHECK(4 == pn_link_credit(snd)); /* Nothing sent yet */
+  d.run();
+  rd = server.delivery;
+  REQUIRE(rd);
+  CHECK(sizeof(rbuf) == pn_link_recv(pn_delivery_link(rd), rbuf, sizeof(rbuf)));
+
+  pn_delivery_abort(sd);
+  d.run();
+  CHECK(pn_delivery_aborted(rd));
+  pn_delivery_settle(rd);
+  /* Abort after sending data consumes credit */
+  CHECK(3 == pn_link_credit(snd));
+  CHECK(3 == pn_link_credit(rcv));
+
+  SEND_RECEIVE_MESSAGE("two", d);
+  CHECK(2 == pn_link_credit(snd));
+  CHECK(2 == pn_link_credit(rcv));
+
+  /* Abort a delivery before any data has been framed, should be dropped. */
+  server.log.clear();
+  sd = pn_delivery(snd, pn_bytes("4"));
+  CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+  pn_delivery_abort(sd);
+  CHECK(pn_link_current(snd) != sd); /* Advanced */
+  d.run();
+  CHECK_THAT(ETYPES(PN_TRANSPORT), Equals(server.log_clear()));
+  /* Aborting wit no frames sent should leave credit untouched */
+  CHECK(2 == pn_link_credit(snd));
+  CHECK(2 == pn_link_credit(rcv));
+
+  SEND_RECEIVE_MESSAGE("three", d);
+  CHECK(1 == pn_link_credit(rcv));
+  CHECK(1 == pn_link_credit(snd));
+}
+
+/* Set capacity and max frame, send a single message */
+static void set_capacity_and_max_frame(size_t capacity, size_t max_frame,
+                                       pn_test::driver_pair &d,
+                                       const char *data) {
+  pn_transport_set_max_frame(d.client.transport, max_frame);
+  pn_connection_open(d.client.connection);
+  pn_session_t *ssn = pn_session(d.client.connection);
+  pn_session_set_incoming_capacity(ssn, capacity);
+  pn_session_open(ssn);
+  pn_link_t *snd = pn_sender(ssn, "x");
+  pn_link_open(snd);
+  d.run();
+  pn_link_flow(d.server.handler.link, 1);
+  d.run();
+  if (pn_transport_closed(d.client.transport) ||
+      pn_transport_closed(d.server.transport))
+    return;
+  /* Send a message */
+  auto_free<pn_message_t, pn_message_free> m(pn_message());
+  pn_message_set_address(m, data);
+  pn_delivery(snd, pn_bytes("x"));
+  pn_message_send(m, snd, NULL);
+  d.run();
+}
+
+/* Test different settings for max-frame, outgoing-window, incoming-capacity */
+TEST_CASE("driver_session_flow_control") {
+  open_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  auto_free<pn_message_t, pn_message_free> m(pn_message());
+  pn_rwbytes_t buf = {0};
+
+  /* Capacity equal to frame size OK */
+  set_capacity_and_max_frame(1234, 1234, d, "foo");
+  pn_delivery_t *dlv = server.delivery;
+  CHECKED_IF(dlv) {
+    message_decode(m, dlv, &buf);
+    CHECK_THAT("foo", Equals(pn_message_get_address(m)));
+  }
+
+  /* Capacity bigger than frame size OK */
+  set_capacity_and_max_frame(12345, 1234, d, "foo");
+  dlv = server.delivery;
+  CHECKED_IF(dlv) {
+    message_decode(m, dlv, &buf);
+    CHECK_THAT("foo", Equals(pn_message_get_address(m)));
+  }
+
+  /* Capacity smaller than frame size is an error */
+  set_capacity_and_max_frame(1234, 12345, d, "foo");
+  CHECK_THAT(
+      *client.last_condition,
+      cond_matches("amqp:internal-error",
+                   "session capacity 1234 is less than frame size 12345"));
+  free(buf.start);
+}
+
+/* Regression test for https://issues.apache.org/jira/browse/PROTON-1832.
+   Make sure we error on attempt to re-attach an already-attached link name.
+   No crash or memory error.
+*/
+TEST_CASE("driver_duplicate_link_server") {
+  open_close_handler client, server;
+  pn_test::driver_pair d(client, server);
+
+  pn_connection_open(d.client.connection);
+  pn_session_t *ssn = pn_session(d.client.connection);
+  pn_session_open(ssn);
+
+  /* Set up link "x" */
+  auto_free<pn_link_t, pn_link_free> x(pn_sender(ssn, "xxx"));
+  pn_link_open(x);
+  d.run();
+  client.log.clear();
+  server.log.clear();
+  /* Free (but don't close) the link and open a new one to generate the invalid
+   * double-attach */
+  pn_link_open(pn_sender(ssn, "xxx"));
+  d.run();
+
+  CHECK_THAT(*pn_transport_condition(d.server.transport),
+             cond_matches("amqp:invalid-field", "xxx"));
+  CHECK_THAT(*pn_connection_remote_condition(d.client.connection),
+             cond_matches("amqp:invalid-field", "xxx"));
+
+  /* Freeing the link at this point is allowed but caused a crash in
+   * transport_unbind with the bug */
+}
+
+/* Reproducer test for https://issues.apache.org/jira/browse/PROTON-1832.
+   Make sure the client does not generate an illegal "attach; attach; detach"
+   sequence from a legal "pn_link_open(); pn_link_close(); pn_link_open()"
+   sequence
+
+   This test is expected to fail till PROTON-1832 is fully fixed
+*/
+TEST_CASE("driver_duplicate_link_client", "[!hide][!shouldfail]") {
+  /* Set up the initial link */
+  open_close_handler client, server;
+  pn_test::driver_pair d(client, server);
+
+  pn_session_t *ssn = pn_session(d.client.connection);
+  pn_session_open(ssn);
+  pn_link_t *x = pn_sender(ssn, "x");
+  pn_link_open(x);
+  d.run();
+  client.log.clear();
+  server.log.clear();
+
+  /* Close the link and open a new link with same name in the same batch of
+   * events. */
+  pn_link_close(x);
+  pn_link_open(pn_sender(ssn, "x"));
+  d.run();
+
+  CHECK_THAT(ETYPES(PN_LINK_REMOTE_CLOSE, PN_LINK_LOCAL_CLOSE, PN_TRANSPORT,
+                    PN_LINK_INIT, PN_LINK_REMOTE_OPEN, PN_LINK_LOCAL_OPEN,
+                    PN_TRANSPORT),
+             Equals(server.log_clear()));
+  CHECK_THAT(*pn_transport_condition(d.server.transport), cond_empty());
+
+  d.run();
+  CHECK_THAT(ETYPES(PN_LINK_LOCAL_CLOSE, PN_TRANSPORT, PN_LINK_REMOTE_CLOSE,
+                    PN_LINK_INIT, PN_LINK_LOCAL_OPEN, PN_TRANSPORT,
+                    PN_LINK_REMOTE_OPEN),
+             Equals(client.log_clear()));
+  CHECK_THAT(*pn_connection_remote_condition(d.client.connection),
+             cond_empty());
+}
+
+/* Settling an incomplete delivery should not cause an error
+   This test will fail till PROTON-1914 is fixed
+*/
+TEST_CASE("driver_settle_incomplete_receiver", "[!hide][!shouldfail]") {
+  send_client_handler client;
+  delivery_handler server;
+  pn_test::driver_pair d(client, server);
+
+  d.run();
+  pn_link_t *rcv = server.link;
+  pn_link_t *snd = client.link;
+  char data[100] = {0};          /* Dummy data to send. */
+  char rbuf[sizeof(data)] = {0}; /* Read buffer for incoming data. */
+  pn_link_flow(rcv, 1);
+  pn_delivery(snd, pn_bytes("1")); /* Prepare to send */
+  d.run();
+
+  /* Send/receive a frame */
+  CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+  d.run();
+  CHECK_THAT(ETYPES(PN_DELIVERY), Equals(server.log_clear()));
+  CHECK(sizeof(data) == pn_link_recv(rcv, rbuf, sizeof(data)));
+  d.run();
+
+  /* Settle the receiver's delivery */
+  pn_delivery_settle(pn_link_current(rcv));
+  d.run();
+  CHECK_THAT(*pn_connection_remote_condition(d.client.connection),
+             cond_empty());
+  CHECK_THAT(*pn_connection_condition(d.server.connection), cond_empty());
+
+  /* Send/receive a frame, should not cause error */
+  CHECK(sizeof(data) == pn_link_send(snd, data, sizeof(data)));
+  d.run();
+  CHECK_THAT(ETYPES(PN_DELIVERY), Equals(server.log_clear()));
+  CHECK(sizeof(data) == pn_link_recv(rcv, rbuf, sizeof(data)));
+  d.run();
+  CHECK_THAT(*pn_connection_remote_condition(d.client.connection),
+             cond_empty());
+  CHECK_THAT(*pn_connection_condition(d.server.connection), cond_empty());
+}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/data.c
----------------------------------------------------------------------
diff --git a/c/tests/data.c b/c/tests/data.c
deleted file mode 100644
index 8f8030f..0000000
--- a/c/tests/data.c
+++ /dev/null
@@ -1,104 +0,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.
- *
- */
-
-#undef NDEBUG                   /* Make sure that assert() is enabled even in a release build. */
-
-#include "test_tools.h"
-#include "core/data.h"
-
-#include <proton/codec.h>
-#include <assert.h>
-#include <stdio.h>
-
-// Make sure we can grow the capacity of a pn_data_t all the way to the max and we stop there.
-static void test_grow(void)
-{
-  pn_data_t* data = pn_data(0);
-  while (pn_data_size(data) < PNI_NID_MAX) {
-    int code = pn_data_put_int(data, 1);
-    if (code) fprintf(stderr, "%d: %s", code, pn_error_text(pn_data_error(data)));
-    assert(code == 0);
-  }
-  assert(pn_data_size(data) == PNI_NID_MAX);
-  int code = pn_data_put_int(data, 1);
-  if (code != PN_OUT_OF_MEMORY)
-    fprintf(stderr, "expected PN_OUT_OF_MEMORY, got  %s\n", pn_code(code));
-  assert(code == PN_OUT_OF_MEMORY);
-  assert(pn_data_size(data) == PNI_NID_MAX);
-  pn_data_free(data);
-}
-
-static void test_multiple(test_t *t) {
-  pn_data_t *data = pn_data(1);
-  pn_data_t *src = pn_data(1);
-
-  /* NULL data pointer */
-  pn_data_fill(data, "M", NULL);
-  TEST_INSPECT(t, "null", data);
-
-  /* Empty data object */
-  pn_data_clear(data);
-  pn_data_fill(data, "M", src);
-  TEST_INSPECT(t, "null", data);
-
-  /* Empty array */
-  pn_data_clear(data);
-  pn_data_clear(src);
-  pn_data_put_array(src, false, PN_SYMBOL);
-  pn_data_fill(data, "M", src);
-  TEST_INSPECT(t, "null", data);
-
-  /* Single-element array */
-  pn_data_clear(data);
-  pn_data_clear(src);
-  pn_data_put_array(src, false, PN_SYMBOL);
-  pn_data_enter(src);
-  pn_data_put_symbol(src, PN_BYTES_LITERAL(foo));
-  pn_data_fill(data, "M", src);
-  TEST_INSPECT(t, ":foo", data);
-
-  /* Multi-element array */
-  pn_data_clear(data);
-  pn_data_clear(src);
-  pn_data_put_array(src, false, PN_SYMBOL);
-  pn_data_enter(src);
-  pn_data_put_symbol(src, PN_BYTES_LITERAL(foo));
-  pn_data_put_symbol(src, PN_BYTES_LITERAL(bar));
-  pn_data_fill(data, "M", src);
-  TEST_INSPECT(t, "@PN_SYMBOL[:foo, :bar]", data);
-
-  /* Non-array */
-  pn_data_clear(data);
-  pn_data_clear(src);
-  pn_data_put_symbol(src, PN_BYTES_LITERAL(baz));
-  pn_data_fill(data, "M", src);
-  TEST_INSPECT(t, ":baz", data);
-
-  pn_data_free(data);
-  pn_data_free(src);
-}
-
-int main(int argc, char **argv) {
-  int failed = 0;
-  test_grow();
-  RUN_ARGV_TEST(failed, t, test_multiple(&t));
-  return failed;
-}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/data_test.cpp
----------------------------------------------------------------------
diff --git a/c/tests/data_test.cpp b/c/tests/data_test.cpp
new file mode 100644
index 0000000..73fcc1f
--- /dev/null
+++ b/c/tests/data_test.cpp
@@ -0,0 +1,92 @@
+/*
+ *
+ * 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 "./pn_test.hpp"
+
+#include "core/data.h"
+
+#include <proton/codec.h>
+#include <proton/error.h>
+
+using namespace pn_test;
+
+// Make sure we can grow the capacity of a pn_data_t all the way to the max and
+// we stop there.
+TEST_CASE("data_grow") {
+  auto_free<pn_data_t, pn_data_free> data(pn_data(0));
+  int code = 0;
+  while (pn_data_size(data) < PNI_NID_MAX && !code) {
+    code = pn_data_put_int(data, 1);
+  }
+  CHECK_THAT(*pn_data_error(data), error_empty());
+  CHECK(pn_data_size(data) == PNI_NID_MAX);
+  code = pn_data_put_int(data, 1);
+  INFO(pn_code(code));
+  CHECK(code == PN_OUT_OF_MEMORY);
+  CHECK(pn_data_size(data) == PNI_NID_MAX);
+}
+
+TEST_CASE("data_multiple") {
+  auto_free<pn_data_t, pn_data_free> data(pn_data(1));
+  auto_free<pn_data_t, pn_data_free> src(pn_data(1));
+
+  /* NULL data pointer */
+  pn_data_fill(data, "M", NULL);
+  CHECK("null" == inspect(data));
+
+  /* Empty data object */
+  pn_data_clear(data);
+  pn_data_fill(data, "M", src.get());
+  CHECK("null" == inspect(data));
+
+  /* Empty array */
+  pn_data_clear(data);
+  pn_data_clear(src);
+  pn_data_put_array(src, false, PN_SYMBOL);
+  pn_data_fill(data, "M", src.get());
+  CHECK("null" == inspect(data));
+
+  /* Single-element array */
+  pn_data_clear(data);
+  pn_data_clear(src);
+  pn_data_put_array(src, false, PN_SYMBOL);
+  pn_data_enter(src);
+  pn_data_put_symbol(src, pn_bytes("foo"));
+  pn_data_fill(data, "M", src.get());
+  CHECK(":foo" == inspect(data));
+
+  /* Multi-element array */
+  pn_data_clear(data);
+  pn_data_clear(src);
+  pn_data_put_array(src, false, PN_SYMBOL);
+  pn_data_enter(src);
+  pn_data_put_symbol(src, pn_bytes("foo"));
+  pn_data_put_symbol(src, pn_bytes("bar"));
+  pn_data_fill(data, "M", src.get());
+  CHECK("@PN_SYMBOL[:foo, :bar]" == inspect(data));
+
+  /* Non-array */
+  pn_data_clear(data);
+  pn_data_clear(src);
+  pn_data_put_symbol(src, pn_bytes("baz"));
+  pn_data_fill(data, "M", src.get());
+  CHECK(":baz" == inspect(data));
+}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/engine.c
----------------------------------------------------------------------
diff --git a/c/tests/engine.c b/c/tests/engine.c
deleted file mode 100644
index 41d17a0..0000000
--- a/c/tests/engine.c
+++ /dev/null
@@ -1,369 +0,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.
- *
- */
-
-#include <stdarg.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-#include <proton/engine.h>
-
-// never remove 'assert()'
-#undef NDEBUG
-#include <assert.h>
-
-// push data from one transport to another
-static int xfer(pn_transport_t *src, pn_transport_t *dest)
-{
-    ssize_t out = pn_transport_pending(src);
-    if (out > 0) {
-        ssize_t in = pn_transport_capacity(dest);
-        if (in > 0) {
-            size_t count = (size_t)((out < in) ? out : in);
-            pn_transport_push(dest,
-                              pn_transport_head(src),
-                              count);
-            pn_transport_pop(src, count);
-            return (int)count;
-        }
-    }
-    return 0;
-}
-
-// transfer all available data between two transports
-static int pump(pn_transport_t *t1, pn_transport_t *t2)
-{
-    int total = 0;
-    int work;
-    do {
-        work = xfer(t1, t2) + xfer(t2, t1);
-        total += work;
-    } while (work);
-    return total;
-}
-
-// handle state changes of the endpoints
-static void process_endpoints(pn_connection_t *conn)
-{
-    pn_session_t *ssn = pn_session_head(conn, PN_LOCAL_UNINIT);
-    while (ssn) {
-        //fprintf(stderr, "Opening session %p\n", (void*)ssn);
-        pn_session_open(ssn);
-        ssn = pn_session_next(ssn, PN_LOCAL_UNINIT);
-    }
-
-    pn_link_t *link = pn_link_head(conn, PN_LOCAL_UNINIT);
-    while (link) {
-        //fprintf(stderr, "Opening link %p\n", (void*)link);
-        pn_link_open(link);
-        link = pn_link_next(link, PN_LOCAL_UNINIT);
-    }
-
-    link = pn_link_head(conn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
-    while (link) {
-        //fprintf(stderr, "Closing link %p\n", (void*)link);
-        pn_link_close(link);
-        link = pn_link_next(link, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
-    }
-
-    ssn = pn_session_head(conn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
-    while (ssn) {
-        //fprintf(stderr, "Closing session %p\n", (void*)ssn);
-        pn_session_close(ssn);
-        ssn = pn_session_next(ssn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
-    }
-}
-
-// bring up a session and a link between the two connections
-static void test_setup(pn_connection_t *c1, pn_transport_t *t1,
-                       pn_connection_t *c2, pn_transport_t *t2)
-{
-    pn_connection_open(c1);
-    pn_connection_open(c2);
-
-    pn_session_t *s1 = pn_session(c1);
-    pn_session_open(s1);
-
-    pn_link_t *tx = pn_sender(s1, "sender");
-    pn_link_open(tx);
-
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-
-    // session and link should be up, c2 should have a receiver link:
-
-    assert(pn_session_state( s1 ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(pn_link_state( tx ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-
-    pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(rx && pn_link_is_receiver(rx));
-}
-
-// test that free'ing the connection should free all contained
-// resources (session, links, deliveries)
-int test_free_connection(int argc, char **argv)
-{
-    fprintf(stdout, "test_free_connection\n");
-    pn_connection_t *c1 = pn_connection();
-    pn_transport_t  *t1 = pn_transport();
-    pn_transport_bind(t1, c1);
-
-    pn_connection_t *c2 = pn_connection();
-    pn_transport_t  *t2 = pn_transport();
-    pn_transport_set_server(t2);
-    pn_transport_bind(t2, c2);
-
-    //pn_transport_trace(t1, PN_TRACE_FRM);
-    test_setup(c1, t1,
-               c2, t2);
-
-    pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(tx);
-    pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(rx);
-
-    // transfer some data across the link:
-    pn_link_flow(rx, 10);
-    pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-    assert(pn_delivery_writable(d1));
-    pn_link_send(tx, "ABC", 4);
-    pn_link_advance(tx);
-
-    // now free the connection, but keep processing the transport
-    process_endpoints(c1);
-    pn_connection_free(c1);
-    while (pump(t1, t2)) {
-        process_endpoints(c2);
-    }
-
-    // delivery should have transfered:
-    assert(pn_link_current(rx) &&
-           pn_delivery_readable(pn_link_current(rx)));
-
-    pn_transport_unbind(t1);
-    pn_transport_free(t1);
-
-    pn_connection_free(c2);
-    pn_transport_unbind(t2);
-    pn_transport_free(t2);
-
-    return 0;
-}
-
-int test_free_session(int argc, char **argv)
-{
-    fprintf(stdout, "test_free_session\n");
-    pn_connection_t *c1 = pn_connection();
-    pn_transport_t  *t1 = pn_transport();
-    pn_transport_bind(t1, c1);
-
-    pn_connection_t *c2 = pn_connection();
-    pn_transport_t  *t2 = pn_transport();
-    pn_transport_set_server(t2);
-    pn_transport_bind(t2, c2);
-
-    //pn_transport_trace(t1, PN_TRACE_FRM);
-    test_setup(c1, t1,
-               c2, t2);
-
-    pn_session_t *ssn = pn_session_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(ssn);
-    pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(tx);
-    pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(rx);
-
-    // prepare for transfer: request some credit
-    pn_link_flow(rx, 10);
-    pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-    assert(pn_delivery_writable(d1));
-
-    // send some data, but also close the session:
-    pn_link_send(tx, "ABC", 4);
-    pn_link_advance(tx);
-
-    pn_session_close(ssn);
-    pn_session_free(ssn);
-
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-
-    // delivery should have transfered:
-    assert(pn_link_current(rx));
-    assert(pn_delivery_readable(pn_link_current(rx)));
-
-    // c2's session should see the close:
-    pn_session_t *ssn2 = pn_session_head(c2, 0);
-    assert(ssn2 && pn_session_state(ssn2) == (PN_LOCAL_CLOSED | PN_REMOTE_CLOSED));
-
-    pn_transport_unbind(t1);
-    pn_transport_free(t1);
-    pn_connection_free(c1);
-
-    pn_transport_unbind(t2);
-    pn_transport_free(t2);
-    pn_connection_free(c2);
-
-    return 0;
-}
-
-int test_free_link(int argc, char **argv)
-{
-    fprintf(stdout, "test_free_link\n");
-    pn_connection_t *c1 = pn_connection();
-    pn_transport_t  *t1 = pn_transport();
-    pn_transport_bind(t1, c1);
-
-    pn_connection_t *c2 = pn_connection();
-    pn_transport_t  *t2 = pn_transport();
-    pn_transport_set_server(t2);
-    pn_transport_bind(t2, c2);
-
-    //pn_transport_trace(t1, PN_TRACE_FRM);
-    test_setup(c1, t1,
-               c2, t2);
-
-    pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(tx);
-    pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(rx);
-
-    // prepare for transfer: request some credit
-    pn_link_flow(rx, 10);
-    pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-    assert(pn_delivery_writable(d1));
-
-    // send some data, then close and destroy the link:
-    pn_link_send(tx, "ABC", 4);
-    pn_link_advance(tx);
-
-    pn_link_close(tx);
-    pn_link_free(tx);
-
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-
-    // the data transfer will complete and the link close
-    // should have been sent to the peer
-    assert(pn_link_current(rx));
-    assert(pn_link_state(rx) == (PN_LOCAL_CLOSED | PN_REMOTE_CLOSED));
-
-    pn_transport_unbind(t1);
-    pn_transport_free(t1);
-    pn_connection_free(c1);
-
-    pn_transport_unbind(t2);
-    pn_transport_free(t2);
-    pn_connection_free(c2);
-
-    return 0;
-}
-
-// regression test fo PROTON-1466 - confusion between links with prefix names
-static int test_link_name_prefix(int argc, char **argv)
-{
-    fprintf(stdout, "test_link_name_prefix\n");
-    pn_connection_t *c1 = pn_connection();
-    pn_transport_t  *t1 = pn_transport();
-    pn_transport_bind(t1, c1);
-
-    pn_connection_t *c2 = pn_connection();
-    pn_transport_t  *t2 = pn_transport();
-    pn_transport_set_server(t2);
-    pn_transport_bind(t2, c2);
-
-    pn_connection_open(c1);
-    pn_connection_open(c2);
-
-    pn_session_t *s1 = pn_session(c1);
-    pn_session_open(s1);
-
-    pn_link_t *l = pn_receiver(s1, "l");
-    pn_link_open(l);
-    pn_link_t *lll = pn_receiver(s1, "lll");
-    pn_link_open(lll);
-    pn_link_t *ll = pn_receiver(s1, "ll");
-    pn_link_open(ll);
-
-    while (pump(t1, t2)) {
-        process_endpoints(c1);
-        process_endpoints(c2);
-    }
-
-    // session and link should be up, c2 should have a receiver link:
-    assert(pn_session_state( s1 ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(pn_link_state( l ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(pn_link_state( lll ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(pn_link_state( ll ) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-
-    pn_link_t *r = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(!strcmp(pn_link_name(r), "l"));
-    r = pn_link_next(r, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(!strcmp(pn_link_name(r), "lll"));
-    r = pn_link_next(r, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
-    assert(!strcmp(pn_link_name(r), "ll"));
-
-    pn_transport_unbind(t1);
-    pn_transport_free(t1);
-    pn_connection_free(c1);
-
-    pn_transport_unbind(t2);
-    pn_transport_free(t2);
-    pn_connection_free(c2);
-
-    return 0;
-}
-
-typedef int (*test_ptr_t)(int argc, char **argv);
-
-test_ptr_t tests[] = {test_free_connection,
-                      test_free_session,
-                      test_free_link,
-                      test_link_name_prefix,
-                      NULL};
-
-int main(int argc, char **argv)
-{
-    test_ptr_t *test = tests;
-    while (*test) {
-        int rc = (*test++)(argc, argv);
-        if (rc)
-            return rc;
-    }
-    return 0;
-}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/engine_test.cpp
----------------------------------------------------------------------
diff --git a/c/tests/engine_test.cpp b/c/tests/engine_test.cpp
new file mode 100644
index 0000000..fa6e23f
--- /dev/null
+++ b/c/tests/engine_test.cpp
@@ -0,0 +1,318 @@
+/*
+ *
+ * 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 "./pn_test.hpp"
+
+#include <proton/engine.h>
+
+// push data from one transport to another
+static int xfer(pn_transport_t *src, pn_transport_t *dest) {
+  ssize_t out = pn_transport_pending(src);
+  if (out > 0) {
+    ssize_t in = pn_transport_capacity(dest);
+    if (in > 0) {
+      size_t count = (size_t)((out < in) ? out : in);
+      pn_transport_push(dest, pn_transport_head(src), count);
+      pn_transport_pop(src, count);
+      return (int)count;
+    }
+  }
+  return 0;
+}
+
+// transfer all available data between two transports
+static int pump(pn_transport_t *t1, pn_transport_t *t2) {
+  int total = 0;
+  int work;
+  do {
+    work = xfer(t1, t2) + xfer(t2, t1);
+    total += work;
+  } while (work);
+  return total;
+}
+
+// handle state changes of the endpoints
+static void process_endpoints(pn_connection_t *conn) {
+  pn_session_t *ssn = pn_session_head(conn, PN_LOCAL_UNINIT);
+  while (ssn) {
+    // fprintf(stderr, "Opening session %p\n", (void*)ssn);
+    pn_session_open(ssn);
+    ssn = pn_session_next(ssn, PN_LOCAL_UNINIT);
+  }
+
+  pn_link_t *link = pn_link_head(conn, PN_LOCAL_UNINIT);
+  while (link) {
+    pn_link_open(link);
+    link = pn_link_next(link, PN_LOCAL_UNINIT);
+  }
+
+  link = pn_link_head(conn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
+  while (link) {
+    pn_link_close(link);
+    link = pn_link_next(link, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
+  }
+
+  ssn = pn_session_head(conn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
+  while (ssn) {
+    pn_session_close(ssn);
+    ssn = pn_session_next(ssn, PN_LOCAL_ACTIVE | PN_REMOTE_CLOSED);
+  }
+}
+
+// bring up a session and a link between the two connections
+static void test_setup(pn_connection_t *c1, pn_transport_t *t1,
+                       pn_connection_t *c2, pn_transport_t *t2) {
+  pn_connection_open(c1);
+  pn_connection_open(c2);
+
+  pn_session_t *s1 = pn_session(c1);
+  pn_session_open(s1);
+
+  pn_link_t *tx = pn_sender(s1, "sender");
+  pn_link_open(tx);
+
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+
+  // session and link should be up, c2 should have a receiver link:
+
+  REQUIRE(pn_session_state(s1) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(pn_link_state(tx) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+
+  pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(rx);
+  REQUIRE(pn_link_is_receiver(rx));
+}
+
+// test that free'ing the connection should free all contained
+// resources (session, links, deliveries)
+TEST_CASE("engine_free_connection") {
+  pn_connection_t *c1 = pn_connection();
+  pn_transport_t *t1 = pn_transport();
+  pn_transport_bind(t1, c1);
+
+  pn_connection_t *c2 = pn_connection();
+  pn_transport_t *t2 = pn_transport();
+  pn_transport_set_server(t2);
+  pn_transport_bind(t2, c2);
+
+  // pn_transport_trace(t1, PN_TRACE_FRM);
+  test_setup(c1, t1, c2, t2);
+
+  pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(tx);
+  pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(rx);
+
+  // transfer some data across the link:
+  pn_link_flow(rx, 10);
+  pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+  REQUIRE(pn_delivery_writable(d1));
+  pn_link_send(tx, "ABC", 4);
+  pn_link_advance(tx);
+
+  // now free the connection, but keep processing the transport
+  process_endpoints(c1);
+  pn_connection_free(c1);
+  while (pump(t1, t2)) {
+    process_endpoints(c2);
+  }
+
+  // delivery should have transfered:
+  REQUIRE(pn_link_current(rx));
+  REQUIRE(pn_delivery_readable(pn_link_current(rx)));
+
+  pn_transport_unbind(t1);
+  pn_transport_free(t1);
+
+  pn_connection_free(c2);
+  pn_transport_unbind(t2);
+  pn_transport_free(t2);
+}
+
+TEST_CASE("engine_free_session") {
+  pn_connection_t *c1 = pn_connection();
+  pn_transport_t *t1 = pn_transport();
+  pn_transport_bind(t1, c1);
+
+  pn_connection_t *c2 = pn_connection();
+  pn_transport_t *t2 = pn_transport();
+  pn_transport_set_server(t2);
+  pn_transport_bind(t2, c2);
+
+  // pn_transport_trace(t1, PN_TRACE_FRM);
+  test_setup(c1, t1, c2, t2);
+
+  pn_session_t *ssn = pn_session_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(ssn);
+  pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(tx);
+  pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(rx);
+
+  // prepare for transfer: request some credit
+  pn_link_flow(rx, 10);
+  pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+  REQUIRE(pn_delivery_writable(d1));
+
+  // send some data, but also close the session:
+  pn_link_send(tx, "ABC", 4);
+  pn_link_advance(tx);
+
+  pn_session_close(ssn);
+  pn_session_free(ssn);
+
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+
+  // delivery should have transfered:
+  REQUIRE(pn_link_current(rx));
+  REQUIRE(pn_delivery_readable(pn_link_current(rx)));
+
+  // c2's session should see the close:
+  pn_session_t *ssn2 = pn_session_head(c2, 0);
+  REQUIRE(ssn2);
+  REQUIRE(pn_session_state(ssn2) == (PN_LOCAL_CLOSED | PN_REMOTE_CLOSED));
+
+  pn_transport_unbind(t1);
+  pn_transport_free(t1);
+  pn_connection_free(c1);
+
+  pn_transport_unbind(t2);
+  pn_transport_free(t2);
+  pn_connection_free(c2);
+}
+
+TEST_CASE("engine_free_link)") {
+  pn_connection_t *c1 = pn_connection();
+  pn_transport_t *t1 = pn_transport();
+  pn_transport_bind(t1, c1);
+
+  pn_connection_t *c2 = pn_connection();
+  pn_transport_t *t2 = pn_transport();
+  pn_transport_set_server(t2);
+  pn_transport_bind(t2, c2);
+
+  // pn_transport_trace(t1, PN_TRACE_FRM);
+  test_setup(c1, t1, c2, t2);
+
+  pn_link_t *tx = pn_link_head(c1, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(tx);
+  pn_link_t *rx = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(rx);
+
+  // prepare for transfer: request some credit
+  pn_link_flow(rx, 10);
+  pn_delivery_t *d1 = pn_delivery(tx, pn_dtag("tag-1", 6));
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+  REQUIRE(pn_delivery_writable(d1));
+
+  // send some data, then close and destroy the link:
+  pn_link_send(tx, "ABC", 4);
+  pn_link_advance(tx);
+
+  pn_link_close(tx);
+  pn_link_free(tx);
+
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+
+  // the data transfer will complete and the link close
+  // should have been sent to the peer
+  REQUIRE(pn_link_current(rx));
+  REQUIRE(pn_link_state(rx) == (PN_LOCAL_CLOSED | PN_REMOTE_CLOSED));
+
+  pn_transport_unbind(t1);
+  pn_transport_free(t1);
+  pn_connection_free(c1);
+
+  pn_transport_unbind(t2);
+  pn_transport_free(t2);
+  pn_connection_free(c2);
+}
+
+// regression test fo PROTON-1466 - confusion between links with prefix names
+TEST_CASE("engine_link_name_prefix)") {
+  pn_connection_t *c1 = pn_connection();
+  pn_transport_t *t1 = pn_transport();
+  pn_transport_bind(t1, c1);
+
+  pn_connection_t *c2 = pn_connection();
+  pn_transport_t *t2 = pn_transport();
+  pn_transport_set_server(t2);
+  pn_transport_bind(t2, c2);
+
+  pn_connection_open(c1);
+  pn_connection_open(c2);
+
+  pn_session_t *s1 = pn_session(c1);
+  pn_session_open(s1);
+
+  pn_link_t *l = pn_receiver(s1, "l");
+  pn_link_open(l);
+  pn_link_t *lll = pn_receiver(s1, "lll");
+  pn_link_open(lll);
+  pn_link_t *ll = pn_receiver(s1, "ll");
+  pn_link_open(ll);
+
+  while (pump(t1, t2)) {
+    process_endpoints(c1);
+    process_endpoints(c2);
+  }
+
+  // session and link should be up, c2 should have a receiver link:
+  REQUIRE(pn_session_state(s1) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(pn_link_state(l) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(pn_link_state(lll) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(pn_link_state(ll) == (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+
+  pn_link_t *r = pn_link_head(c2, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(std::string("l") == pn_link_name(r));
+  r = pn_link_next(r, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(std::string("lll") == pn_link_name(r));
+  r = pn_link_next(r, (PN_LOCAL_ACTIVE | PN_REMOTE_ACTIVE));
+  REQUIRE(std::string("ll") == pn_link_name(r));
+
+  pn_transport_unbind(t1);
+  pn_transport_free(t1);
+  pn_connection_free(c1);
+
+  pn_transport_unbind(t2);
+  pn_transport_free(t2);
+  pn_connection_free(c2);
+}

http://git-wip-us.apache.org/repos/asf/qpid-proton/blob/0bdba37d/c/tests/event.c
----------------------------------------------------------------------
diff --git a/c/tests/event.c b/c/tests/event.c
deleted file mode 100644
index 6746c3b..0000000
--- a/c/tests/event.c
+++ /dev/null
@@ -1,109 +0,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.
- *
- */
-
-#include <proton/object.h>
-#include <proton/event.h>
-#include <stdlib.h>
-
-#define assert(E) ((E) ? 0 : (abort(), 0))
-
-static void test_collector(void) {
-  pn_collector_t *collector = pn_collector();
-  assert(collector);
-  pn_free(collector);
-}
-
-#define SETUP_COLLECTOR \
-  void *obj = pn_class_new(PN_OBJECT, 0); \
-  pn_collector_t *collector = pn_collector(); \
-  assert(collector); \
-  pn_event_t *event = pn_collector_put(collector, PN_OBJECT, obj, (pn_event_type_t) 0); \
-  pn_decref(obj); \
-
-static void test_collector_put(void) {
-  SETUP_COLLECTOR;
-  assert(event);
-  assert(pn_event_context(event) == obj);
-  pn_free(collector);
-}
-
-static void test_collector_peek(void) {
-  SETUP_COLLECTOR;
-  pn_event_t *head = pn_collector_peek(collector);
-  assert(head == event);
-  pn_free(collector);
-}
-
-static void test_collector_pop(void) {
-  SETUP_COLLECTOR;
-  pn_event_t *head = pn_collector_peek(collector);
-  assert(head == event);
-  pn_collector_pop(collector);
-  head = pn_collector_peek(collector);
-  assert(!head);
-  pn_free(collector);
-}
-
-static void test_collector_pool(void) {
-  SETUP_COLLECTOR;
-  pn_event_t *head = pn_collector_peek(collector);
-  assert(head == event);
-  pn_collector_pop(collector);
-  head = pn_collector_peek(collector);
-  assert(!head);
-  void *obj2 = pn_class_new(PN_OBJECT, 0);
-  pn_event_t *event2 = pn_collector_put(collector, PN_OBJECT, obj2, (pn_event_type_t) 0);
-  pn_decref(obj2);
-  assert(event == event2);
-  pn_free(collector);
-}
-
-static void test_event_incref(bool eventfirst) {
-  SETUP_COLLECTOR;
-  pn_event_t *head = pn_collector_peek(collector);
-  assert(head == event);
-  pn_incref(head);
-  pn_collector_pop(collector);
-  assert(!pn_collector_peek(collector));
-  void *obj2 = pn_class_new(PN_OBJECT, 0);
-  pn_event_t *event2 = pn_collector_put(collector, PN_OBJECT, obj2, (pn_event_type_t) 0);
-  pn_decref(obj2);
-  assert(head != event2);
-  if (eventfirst) {
-    pn_decref(head);
-    pn_free(collector);
-  } else {
-    pn_free(collector);
-    pn_decref(head);
-  }
-}
-
-int main(int argc, char **argv)
-{
-  test_collector();
-  test_collector_put();
-  test_collector_peek();
-  test_collector_pop();
-  test_collector_pool();
-  test_event_incref(true);
-  test_event_incref(false);
-  return 0;
-}


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