You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by cc...@apache.org on 2016/09/06 20:58:50 UTC

incubator-mynewt-core git commit: MYNEWT-347 BLE Host - GAP update; use L2CAP proc.

Repository: incubator-mynewt-core
Updated Branches:
  refs/heads/develop a1a314579 -> a3704587c


MYNEWT-347 BLE Host - GAP update; use L2CAP proc.

Prior to this change, the host did not failover to the L2CAP connection
parameter update procedure when the GAP update procedure failed.
Rather, it was up to the application to manually do that.

Now, if the host automatically performs the L2CAP procedure if all of
the following are true:
    * We are the slave.
    * Controller reports the following status code in its command
      status event:
        o Unknown HCI command (0x01).
    * Controller reports the following status code in its LE connection
      update complete event:
        o Unsupported remote feature (0x1a).

In addition, there was some missing functionality that this commit fills
in:
    * Timeouts for GAP update procedures (30 seconds).
    * Report and delete outstanding L2CAP sig procedures when the
      connection is broken.


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/commit/a3704587
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/tree/a3704587
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/diff/a3704587

Branch: refs/heads/develop
Commit: a3704587c40893b0021a099a9b451653f16c17d6
Parents: a1a3145
Author: Christopher Collins <cc...@apache.org>
Authored: Mon Sep 5 11:10:30 2016 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Tue Sep 6 13:29:40 2016 -0700

----------------------------------------------------------------------
 apps/bletiny/src/main.c                     |   5 +-
 libs/os/include/os/os_time.h                |   5 +-
 net/nimble/host/include/host/ble_gap.h      |   1 +
 net/nimble/host/include/host/ble_l2cap.h    |   3 +-
 net/nimble/host/src/ble_gap.c               | 394 ++++++++++++++++++++--
 net/nimble/host/src/ble_hs_conn_priv.h      |   1 -
 net/nimble/host/src/ble_l2cap_sig.c         |  34 +-
 net/nimble/host/src/ble_l2cap_sig_priv.h    |   1 +
 net/nimble/host/src/test/ble_gap_test.c     | 411 +++++++++++++++++++----
 net/nimble/host/src/test/ble_hs_test_util.c | 112 +++++-
 net/nimble/host/src/test/ble_hs_test_util.h |  14 +
 net/nimble/host/src/test/ble_l2cap_test.c   | 135 +-------
 12 files changed, 893 insertions(+), 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/apps/bletiny/src/main.c
----------------------------------------------------------------------
diff --git a/apps/bletiny/src/main.c b/apps/bletiny/src/main.c
index d06fa0a..dfd0aee 100755
--- a/apps/bletiny/src/main.c
+++ b/apps/bletiny/src/main.c
@@ -1044,9 +1044,10 @@ bletiny_gap_event(struct ble_gap_event *event, void *arg)
 }
 
 static void
-bletiny_on_l2cap_update(int status, void *arg)
+bletiny_on_l2cap_update(uint16_t conn_handle, int status, void *arg)
 {
-    console_printf("l2cap update complete; status=%d\n", status);
+    console_printf("l2cap update complete; conn_handle=%d status=%d\n",
+                   conn_handle, status);
 }
 
 static void

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/libs/os/include/os/os_time.h
----------------------------------------------------------------------
diff --git a/libs/os/include/os/os_time.h b/libs/os/include/os/os_time.h
index 7e577eb..38b6bd9 100644
--- a/libs/os/include/os/os_time.h
+++ b/libs/os/include/os/os_time.h
@@ -55,11 +55,12 @@
 #include <stdbool.h>
 #include <stdint.h>
 
-typedef uint32_t os_time_t;
-
 #ifndef UINT32_MAX
 #define UINT32_MAX  0xFFFFFFFFU
 #endif
+
+typedef uint32_t os_time_t;
+#define OS_TIME_MAX UINT32_MAX
  
 /* Used to wait forever for events and mutexs */
 #define OS_TIMEOUT_NEVER    (UINT32_MAX)

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/include/host/ble_gap.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/include/host/ble_gap.h b/net/nimble/host/include/host/ble_gap.h
index 960ad7b..f61324b 100644
--- a/net/nimble/host/include/host/ble_gap.h
+++ b/net/nimble/host/include/host/ble_gap.h
@@ -555,6 +555,7 @@ int ble_gap_wl_set(const struct ble_gap_white_entry *white_list,
                    uint8_t white_list_count);
 int ble_gap_update_params(uint16_t conn_handle,
                           const struct ble_gap_upd_params *params);
+int ble_gap_dbg_update_active(uint16_t conn_handle);
 int ble_gap_security_initiate(uint16_t conn_handle);
 int ble_gap_pair_initiate(uint16_t conn_handle);
 int ble_gap_encryption_initiate(uint16_t conn_handle, const uint8_t *ltk,

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/include/host/ble_l2cap.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/include/host/ble_l2cap.h b/net/nimble/host/include/host/ble_l2cap.h
index 620e0eb..ebdfa1c 100644
--- a/net/nimble/host/include/host/ble_l2cap.h
+++ b/net/nimble/host/include/host/ble_l2cap.h
@@ -52,7 +52,8 @@ struct ble_hs_conn;
 #define BLE_L2CAP_SIG_ERR_MTU_EXCEEDED          0x0001
 #define BLE_L2CAP_SIG_ERR_INVALID_CID           0x0002
 
-typedef void ble_l2cap_sig_update_fn(int status, void *arg);
+typedef void ble_l2cap_sig_update_fn(uint16_t conn_handle, int status,
+                                     void *arg);
 
 struct ble_l2cap_sig_update_params {
     uint16_t itvl_min;

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/ble_gap.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_gap.c b/net/nimble/host/src/ble_gap.c
index f53a738..2699e72 100644
--- a/net/nimble/host/src/ble_gap.c
+++ b/net/nimble/host/src/ble_gap.c
@@ -22,6 +22,7 @@
 #include <errno.h>
 #include "bsp/bsp.h"
 #include "os/os.h"
+#include "util/mem.h"
 #include "nimble/nimble_opt.h"
 #include "host/ble_hs_adv.h"
 #include "ble_hs_priv.h"
@@ -59,16 +60,18 @@
  */
 
 /** GAP procedure op codes. */
-#define BLE_GAP_OP_NULL                                 0
-#define BLE_GAP_OP_M_DISC                               1
-#define BLE_GAP_OP_M_CONN                               2
-#define BLE_GAP_OP_S_ADV                                1
+#define BLE_GAP_OP_NULL                         0
+#define BLE_GAP_OP_M_DISC                       1
+#define BLE_GAP_OP_M_CONN                       2
+#define BLE_GAP_OP_S_ADV                        1
 
 /**
  * If an attempt to cancel an active procedure fails, the attempt is retried
  * at this rate (ms).
  */
-#define BLE_GAP_CANCEL_RETRY_RATE                              100 /* ms */
+#define BLE_GAP_CANCEL_RETRY_RATE               100 /* ms */
+
+#define BLE_GAP_UPDATE_TIMEOUT                  (30 * OS_TICKS_PER_SEC)
 
 /**
  * The maximum amount of user data that can be put into the advertising data.
@@ -78,6 +81,8 @@
 #define BLE_GAP_ADV_DATA_LIMIT_FLAGS    (BLE_HCI_MAX_ADV_DATA_LEN - 3)
 #define BLE_GAP_ADV_DATA_LIMIT_NO_FLAGS BLE_HCI_MAX_ADV_DATA_LEN
 
+#define BLE_GAP_MAX_UPDATE_ENTRIES      1
+
 static const struct ble_gap_conn_params ble_gap_conn_params_dflt = {
     .scan_itvl = 0x0010,
     .scan_window = 0x0010,
@@ -141,9 +146,13 @@ static bssnz_t struct {
     unsigned adv_auto_flags:1;
 } ble_gap_slave;
 
-static int ble_gap_adv_enable_tx(int enable);
-static int ble_gap_conn_cancel_tx(void);
-static int ble_gap_disc_enable_tx(int enable, int filter_duplicates);
+struct ble_gap_update_entry {
+    SLIST_ENTRY(ble_gap_update_entry) next;
+    struct ble_gap_upd_params params;
+    os_time_t exp_os_ticks;
+    uint16_t conn_handle;
+};
+SLIST_HEAD(ble_gap_update_entry_list, ble_gap_update_entry);
 
 struct ble_gap_snapshot {
     struct ble_gap_conn_desc *desc;
@@ -151,6 +160,23 @@ struct ble_gap_snapshot {
     void *cb_arg;
 };
 
+static void *ble_gap_update_entry_mem;
+static struct os_mempool ble_gap_update_entry_pool;
+static struct ble_gap_update_entry_list ble_gap_update_entries;
+
+static void ble_gap_update_entry_free(struct ble_gap_update_entry *entry);
+static struct ble_gap_update_entry *
+ble_gap_update_entry_find(uint16_t conn_handle,
+                          struct ble_gap_update_entry **out_prev);
+static struct ble_gap_update_entry *
+ble_gap_update_entry_remove(uint16_t conn_handle);
+static void
+ble_gap_update_l2cap_cb(uint16_t conn_handle, int status, void *arg);
+
+static int ble_gap_adv_enable_tx(int enable);
+static int ble_gap_conn_cancel_tx(void);
+static int ble_gap_disc_enable_tx(int enable, int filter_duplicates);
+
 STATS_SECT_DECL(ble_gap_stats) ble_gap_stats;
 STATS_NAME_START(ble_gap_stats)
     STATS_NAME(ble_gap_stats, wl_set)
@@ -187,6 +213,24 @@ STATS_NAME_START(ble_gap_stats)
 STATS_NAME_END(ble_gap_stats)
 
 /*****************************************************************************
+ * $debug                                                                    *
+ *****************************************************************************/
+
+#if BLE_HS_DEBUG
+int
+ble_gap_dbg_update_active(uint16_t conn_handle)
+{
+    const struct ble_gap_update_entry *entry;
+
+    ble_hs_lock();
+    entry = ble_gap_update_entry_find(conn_handle, NULL);
+    ble_hs_unlock();
+
+    return entry != NULL;
+}
+#endif
+
+/*****************************************************************************
  * $log                                                                      *
  *****************************************************************************/
 
@@ -606,6 +650,11 @@ ble_gap_update_notify(uint16_t conn_handle, int status)
     event.conn_update.status = status;
 
     ble_gap_call_conn_event_cb(&event, conn_handle);
+
+    /* Terminate the connection on procedure timeout. */
+    if (status == BLE_HS_ETIMEOUT) {
+        ble_gap_terminate(conn_handle, BLE_ERR_REM_USER_CONN_TERM);
+    }
 }
 
 static uint32_t
@@ -648,16 +697,62 @@ ble_gap_slave_ticks_until_exp(void)
     return 0;
 }
 
+static uint16_t
+ble_gap_update_next_exp(int32_t *out_ticks_from_now)
+{
+    struct ble_gap_update_entry *entry;
+    os_time_t now;
+    uint16_t conn_handle;
+    int32_t best_ticks;
+    int32_t ticks;
+
+    BLE_HS_DBG_ASSERT(ble_hs_locked_by_cur_task());
+
+    conn_handle = BLE_HS_CONN_HANDLE_NONE;
+    best_ticks = BLE_HS_FOREVER;
+    now = os_time_get();
+
+    SLIST_FOREACH(entry, &ble_gap_update_entries, next) {
+        ticks = entry->exp_os_ticks - now;
+        if (ticks <= 0) {
+            ticks = 0;
+        }
+
+        if (ticks < best_ticks) {
+            conn_handle = entry->conn_handle;
+            best_ticks = ticks;
+        }
+    }
+
+    if (out_ticks_from_now != NULL) {
+        *out_ticks_from_now = best_ticks;
+    }
+
+    return conn_handle;
+
+}
+
+static uint32_t
+ble_gap_update_ticks_until_exp(void)
+{
+    int32_t ticks;
+
+    ble_gap_update_next_exp(&ticks);
+    return ticks;
+}
+
 static void
 ble_gap_heartbeat_sched(void)
 {
     int32_t mst_ticks;
     int32_t slv_ticks;
+    int32_t upd_ticks;
     int32_t ticks;
 
     mst_ticks = ble_gap_master_ticks_until_exp();
     slv_ticks = ble_gap_slave_ticks_until_exp();
-    ticks = min(mst_ticks, slv_ticks);
+    upd_ticks = ble_gap_update_ticks_until_exp();
+    ticks = min(min(mst_ticks, slv_ticks), upd_ticks);
 
     ble_hs_heartbeat_sched(ticks);
 }
@@ -702,14 +797,23 @@ ble_gap_master_failed(int status)
 static void
 ble_gap_update_failed(uint16_t conn_handle, int status)
 {
+    struct ble_gap_update_entry *entry;
+
     STATS_INC(ble_gap_stats, update_fail);
-    ble_hs_atomic_conn_set_flags(conn_handle, BLE_HS_CONN_F_UPDATE, 0);
+
+    ble_hs_lock();
+    entry = ble_gap_update_entry_remove(conn_handle);
+    ble_hs_unlock();
+
+    ble_gap_update_entry_free(entry);
+
     ble_gap_update_notify(conn_handle, status);
 }
 
 void
 ble_gap_conn_broken(uint16_t conn_handle, int reason)
 {
+    struct ble_gap_update_entry *entry;
     struct ble_gap_snapshot snap;
     struct ble_gap_event event;
     int rc;
@@ -723,10 +827,21 @@ ble_gap_conn_broken(uint16_t conn_handle, int reason)
         return;
     }
 
+    /* If there was a connection update in progress, indicate to the
+     * application that it did not complete.
+     */
+    ble_hs_lock();
+    entry = ble_gap_update_entry_remove(conn_handle);
+    ble_hs_unlock();
+
+    ble_gap_update_notify(conn_handle, reason);
+    ble_gap_update_entry_free(entry);
+
     /* Indicate the connection termination to each module.  The order matters
      * here: gatts must come before gattc to ensure the application does not
      * get informed of spurious notify-tx events.
      */
+    ble_l2cap_sig_conn_broken(conn_handle, reason);
     ble_sm_connection_broken(conn_handle);
     ble_gatts_connection_broken(conn_handle);
     ble_gattc_connection_broken(conn_handle);
@@ -740,6 +855,16 @@ ble_gap_conn_broken(uint16_t conn_handle, int reason)
     STATS_INC(ble_gap_stats, disconnect);
 }
 
+static void
+ble_gap_update_to_l2cap(const struct ble_gap_upd_params *params,
+                        struct ble_l2cap_sig_update_params *l2cap_params)
+{
+    l2cap_params->itvl_min = params->itvl_min;
+    l2cap_params->itvl_max = params->itvl_max;
+    l2cap_params->slave_latency = params->latency;
+    l2cap_params->timeout_multiplier = params->supervision_timeout;
+}
+
 void
 ble_gap_rx_disconn_complete(struct hci_disconn_complete *evt)
 {
@@ -766,36 +891,79 @@ ble_gap_rx_disconn_complete(struct hci_disconn_complete *evt)
 void
 ble_gap_rx_update_complete(struct hci_le_conn_upd_complete *evt)
 {
+    struct ble_gap_update_entry *entry;
+
 #if !NIMBLE_OPT(CONNECT)
     return;
 #endif
 
+    struct ble_l2cap_sig_update_params l2cap_params;
     struct ble_gap_event event;
     struct ble_hs_conn *conn;
+    int cb_status;
+    int call_cb;
+    int rc;
 
     STATS_INC(ble_gap_stats, rx_update_complete);
 
     memset(&event, 0, sizeof event);
+    memset(&l2cap_params, 0, sizeof l2cap_params);
 
     ble_hs_lock();
 
     conn = ble_hs_conn_find(evt->connection_handle);
     if (conn != NULL) {
-        if (evt->status == 0) {
+        switch (evt->status) {
+        case 0:
+            /* Connection successfully updated. */
             conn->bhc_itvl = evt->conn_itvl;
             conn->bhc_latency = evt->conn_latency;
             conn->bhc_supervision_timeout = evt->supervision_timeout;
+            break;
+
+        case BLE_ERR_UNSUPP_REM_FEATURE:
+            /* Peer reports that it doesn't support the procedure.  This should
+             * only happen if our controller sent the 4.1 Connection Parameters
+             * Request Procedure.  If we are the slave, fail over to the L2CAP
+             * update procedure.
+             */
+            entry = ble_gap_update_entry_find(evt->connection_handle, NULL);
+            if (entry != NULL && !(conn->bhc_flags & BLE_HS_CONN_F_MASTER)) {
+                ble_gap_update_to_l2cap(&entry->params, &l2cap_params);
+            }
+            break;
+
+        default:
+            break;
         }
     }
 
-    conn->bhc_flags &= ~BLE_HS_CONN_F_UPDATE;
+    /* We aren't failing over to L2CAP, the update procedure is complete. */
+    if (l2cap_params.itvl_min == 0) {
+        entry = ble_gap_update_entry_remove(evt->connection_handle);
+        ble_gap_update_entry_free(entry);
+    }
 
     ble_hs_unlock();
 
-    event.type = BLE_GAP_EVENT_CONN_UPDATE;
-    event.conn_update.conn_handle = evt->connection_handle;
-    event.conn_update.status = BLE_HS_HCI_ERR(evt->status);
-    ble_gap_call_conn_event_cb(&event, evt->connection_handle);
+    if (l2cap_params.itvl_min != 0) {
+        rc = ble_l2cap_sig_update(evt->connection_handle,
+                                  &l2cap_params,
+                                  ble_gap_update_l2cap_cb, NULL);
+        if (rc == 0) {
+            call_cb = 0;
+        } else {
+            call_cb = 1;
+            cb_status = rc;
+        }
+    } else {
+        call_cb = 1;
+        cb_status = BLE_HS_HCI_ERR(evt->status);
+    }
+
+    if (call_cb) {
+        ble_gap_update_notify(evt->connection_handle, cb_status);
+    }
 }
 
 /**
@@ -1156,6 +1324,34 @@ ble_gap_slave_heartbeat(void)
     return BLE_HS_FOREVER;
 }
 
+static int32_t
+ble_gap_update_heartbeat(void)
+{
+    struct ble_gap_update_entry *entry;
+    int32_t ticks_until_exp;
+    uint16_t conn_handle;
+
+    do {
+        ble_hs_lock();
+
+        conn_handle = ble_gap_update_next_exp(&ticks_until_exp);
+        if (ticks_until_exp == 0) {
+            entry = ble_gap_update_entry_remove(conn_handle);
+        } else {
+            entry = NULL;
+        }
+
+        ble_hs_unlock();
+
+        if (entry != NULL) {
+            ble_gap_update_notify(conn_handle, BLE_HS_ETIMEOUT);
+            ble_gap_update_entry_free(entry);
+        }
+    } while (entry != NULL);
+
+    return ticks_until_exp;
+}
+
 /**
  * Handles timed-out master procedures.
  *
@@ -1167,13 +1363,15 @@ ble_gap_slave_heartbeat(void)
 int32_t
 ble_gap_heartbeat(void)
 {
+    int32_t update_ticks;
     int32_t master_ticks;
     int32_t slave_ticks;
 
     master_ticks = ble_gap_master_heartbeat();
     slave_ticks = ble_gap_slave_heartbeat();
+    update_ticks = ble_gap_update_heartbeat();
 
-    return min(master_ticks, slave_ticks);
+    return min(min(master_ticks, slave_ticks), update_ticks);
 }
 
 /*****************************************************************************
@@ -2450,6 +2648,93 @@ done:
  * $update connection parameters                                             *
  *****************************************************************************/
 
+static struct ble_gap_update_entry *
+ble_gap_update_entry_alloc(void)
+{
+    struct ble_gap_update_entry *entry;
+
+    entry = os_memblock_get(&ble_gap_update_entry_pool);
+    if (entry != NULL) {
+        memset(entry, 0, sizeof *entry);
+    }
+
+    return entry;
+}
+
+static void
+ble_gap_update_entry_free(struct ble_gap_update_entry *entry)
+{
+    int rc;
+
+    if (entry != NULL) {
+        rc = os_memblock_put(&ble_gap_update_entry_pool, entry);
+        BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+    }
+}
+
+static struct ble_gap_update_entry *
+ble_gap_update_entry_find(uint16_t conn_handle,
+                          struct ble_gap_update_entry **out_prev)
+{
+    struct ble_gap_update_entry *entry;
+    struct ble_gap_update_entry *prev;
+
+    BLE_HS_DBG_ASSERT(ble_hs_locked_by_cur_task());
+
+    prev = NULL;
+    SLIST_FOREACH(entry, &ble_gap_update_entries, next) {
+        if (entry->conn_handle == conn_handle) {
+            break;
+        }
+
+        prev = entry;
+    }
+
+    if (out_prev != NULL) {
+        *out_prev = prev;
+    }
+
+    return entry;
+}
+
+static struct ble_gap_update_entry *
+ble_gap_update_entry_remove(uint16_t conn_handle)
+{
+    struct ble_gap_update_entry *entry;
+    struct ble_gap_update_entry *prev;
+
+    entry = ble_gap_update_entry_find(conn_handle, &prev);
+    if (entry != NULL) {
+        if (prev == NULL) {
+            SLIST_REMOVE_HEAD(&ble_gap_update_entries, next);
+        } else {
+            SLIST_NEXT(prev, next) = SLIST_NEXT(entry, next);
+        }
+    }
+
+    return entry;
+}
+
+static void
+ble_gap_update_l2cap_cb(uint16_t conn_handle, int status, void *arg)
+{
+    struct ble_gap_update_entry *entry;
+
+    /* Report failures and rejections.  Success gets reported when the
+     * controller sends the connection update complete event.
+     */
+    if (status != 0) {
+        ble_hs_lock();
+        entry = ble_gap_update_entry_remove(conn_handle);
+        ble_hs_unlock();
+
+        if (entry != NULL) {
+            ble_gap_update_entry_free(entry);
+            ble_gap_update_notify(conn_handle, status);
+        }
+    }
+}
+
 static int
 ble_gap_tx_param_pos_reply(uint16_t conn_handle,
                            struct ble_gap_upd_params *params)
@@ -2538,9 +2823,6 @@ ble_gap_rx_param_req(struct hci_le_conn_param_req *evt)
         rc = ble_gap_tx_param_pos_reply(evt->connection_handle, &self_params);
         if (rc != 0) {
             ble_gap_update_failed(evt->connection_handle, rc);
-        } else {
-            ble_hs_atomic_conn_set_flags(evt->connection_handle,
-                                         BLE_HS_CONN_F_UPDATE, 1);
         }
     } else {
         ble_gap_tx_param_neg_reply(evt->connection_handle, reject_reason);
@@ -2600,10 +2882,14 @@ ble_gap_update_params(uint16_t conn_handle,
     return BLE_HS_ENOTSUP;
 #endif
 
+    struct ble_l2cap_sig_update_params l2cap_params;
+    struct ble_gap_update_entry *entry;
     struct ble_hs_conn *conn;
     int rc;
 
     STATS_INC(ble_gap_stats, update);
+    memset(&l2cap_params, 0, sizeof l2cap_params);
+    entry = NULL;
 
     ble_hs_lock();
 
@@ -2613,28 +2899,58 @@ ble_gap_update_params(uint16_t conn_handle,
         goto done;
     }
 
-    if (conn->bhc_flags & BLE_HS_CONN_F_UPDATE) {
+    entry = ble_gap_update_entry_find(conn_handle, NULL);
+    if (entry != NULL) {
         rc = BLE_HS_EALREADY;
         goto done;
     }
 
+    entry = ble_gap_update_entry_alloc();
+    if (entry == NULL) {
+        rc = BLE_HS_ENOMEM;
+        goto done;
+    }
+
+    entry->conn_handle = conn_handle;
+    entry->params = *params;
+    entry->exp_os_ticks = os_time_get() + BLE_GAP_UPDATE_TIMEOUT;
+
     BLE_HS_LOG(INFO, "GAP procedure initiated: ");
     ble_gap_log_update(conn_handle, params);
     BLE_HS_LOG(INFO, "\n");
 
     rc = ble_gap_update_tx(conn_handle, params);
-    if (rc != 0) {
-        goto done;
-    }
 
-    conn->bhc_flags |= BLE_HS_CONN_F_UPDATE;
+    /* If our controller reports that it doesn't support the update procedure,
+     * and we are the slave, fail over to the L2CAP update procedure.
+     */
+    if (rc == BLE_HS_HCI_ERR(BLE_ERR_UNKNOWN_HCI_CMD) &&
+        !(conn->bhc_flags & BLE_HS_CONN_F_MASTER)) {
+
+        ble_gap_update_to_l2cap(params, &l2cap_params);
+    }
 
 done:
     ble_hs_unlock();
 
     if (rc != 0) {
+        ble_gap_update_entry_free(entry);
+
+        if (l2cap_params.itvl_min != 0) {
+            rc = ble_l2cap_sig_update(conn_handle,
+                                      &l2cap_params,
+                                      ble_gap_update_l2cap_cb, NULL);
+        }
+    }
+
+    ble_hs_lock();
+    if (rc == 0) {
+        SLIST_INSERT_HEAD(&ble_gap_update_entries, entry, next);
+    } else {
         STATS_INC(ble_gap_stats, update_fail);
     }
+    ble_hs_unlock();
+
     return rc;
 }
 
@@ -2927,15 +3243,41 @@ ble_gap_init(void)
 {
     int rc;
 
+    free(ble_gap_update_entry_mem);
+
     memset(&ble_gap_master, 0, sizeof ble_gap_master);
     memset(&ble_gap_slave, 0, sizeof ble_gap_slave);
 
+    SLIST_INIT(&ble_gap_update_entries);
+
+    rc = mem_malloc_mempool(&ble_gap_update_entry_pool,
+                            BLE_GAP_MAX_UPDATE_ENTRIES,
+                            sizeof (struct ble_gap_update_entry),
+                            "ble_gap_update",
+                            &ble_gap_update_entry_mem);
+    switch (rc) {
+    case 0:
+        break;
+    case OS_ENOMEM:
+        rc = BLE_HS_ENOMEM;
+        goto err;
+    default:
+        rc = BLE_HS_EOS;
+        goto err;
+    }
+
     rc = stats_init_and_reg(
         STATS_HDR(ble_gap_stats), STATS_SIZE_INIT_PARMS(ble_gap_stats,
         STATS_SIZE_32), STATS_NAME_INIT_PARMS(ble_gap_stats), "ble_gap");
     if (rc != 0) {
-        return BLE_HS_EOS;
+        goto err;
     }
 
     return 0;
+
+err:
+    free(ble_gap_update_entry_mem);
+    ble_gap_update_entry_mem = NULL;
+
+    return rc;
 }

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/ble_hs_conn_priv.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hs_conn_priv.h b/net/nimble/host/src/ble_hs_conn_priv.h
index da8d2fc..ac87ff5 100644
--- a/net/nimble/host/src/ble_hs_conn_priv.h
+++ b/net/nimble/host/src/ble_hs_conn_priv.h
@@ -31,7 +31,6 @@ struct ble_l2cap_chan;
 typedef uint8_t ble_hs_conn_flags_t;
 
 #define BLE_HS_CONN_F_MASTER        0x01
-#define BLE_HS_CONN_F_UPDATE        0x02
 
 struct ble_hs_conn {
     SLIST_ENTRY(ble_hs_conn) bhc_next;

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/ble_l2cap_sig.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_l2cap_sig.c b/net/nimble/host/src/ble_l2cap_sig.c
index a382b09..7b92e14 100644
--- a/net/nimble/host/src/ble_l2cap_sig.c
+++ b/net/nimble/host/src/ble_l2cap_sig.c
@@ -213,7 +213,7 @@ ble_l2cap_sig_proc_matches(struct ble_l2cap_sig_proc *proc,
         return 0;
     }
 
-    if (id != proc->id) {
+    if (id != 0 && id != proc->id) {
         return 0;
     }
 
@@ -227,7 +227,8 @@ ble_l2cap_sig_proc_matches(struct ble_l2cap_sig_proc *proc,
  *
  * @param conn_handle           The connection handle to match against.
  * @param op                    The op code to match against.
- * @param identifier            The identifier to match against.
+ * @param identifier            The identifier to match against;
+ *                                  0=ignore this criterion.
  *
  * @return                      The matching proc entry on success;
  *                                  null on failure.
@@ -280,7 +281,7 @@ ble_l2cap_sig_update_call_cb(struct ble_l2cap_sig_proc *proc, int status)
     }
 
     if (proc->update.cb != NULL) {
-        proc->update.cb(status, proc->update.cb_arg);
+        proc->update.cb(proc->conn_handle, status, proc->update.cb_arg);
     }
 }
 
@@ -561,6 +562,24 @@ ble_l2cap_sig_extract_expired(struct ble_l2cap_sig_proc_list *dst_list)
     ble_hs_unlock();
 }
 
+void
+ble_l2cap_sig_conn_broken(uint16_t conn_handle, int reason)
+{
+    struct ble_l2cap_sig_proc *proc;
+
+    /* If there was a connection update in progress, indicate to the
+     * application that it did not complete.
+     */
+
+    proc = ble_l2cap_sig_proc_extract(conn_handle,
+                                      BLE_L2CAP_SIG_PROC_OP_UPDATE, 0);
+
+    if (proc != NULL) {
+        ble_l2cap_sig_update_call_cb(proc, reason);
+        ble_l2cap_sig_proc_free(proc);
+    }
+}
+
 /**
  * Applies periodic checks and actions to all active procedures.
  *
@@ -584,10 +603,13 @@ ble_l2cap_sig_heartbeat(void)
      */
     ble_l2cap_sig_extract_expired(&temp_list);
 
-    /* Terminate the connection associated with each timed-out procedure. */
-    STAILQ_FOREACH(proc, &temp_list, next) {
+    /* Report a failure for each timed out procedure. */
+    while ((proc = STAILQ_FIRST(&temp_list)) != NULL) {
         STATS_INC(ble_l2cap_stats, proc_timeout);
-        ble_gap_terminate(proc->conn_handle, BLE_ERR_REM_USER_CONN_TERM);
+        ble_l2cap_sig_update_call_cb(proc, BLE_HS_ETIMEOUT);
+
+        STAILQ_REMOVE_HEAD(&temp_list, next);
+        ble_l2cap_sig_proc_free(proc);
     }
 
     return BLE_HS_FOREVER;

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/ble_l2cap_sig_priv.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_l2cap_sig_priv.h b/net/nimble/host/src/ble_l2cap_sig_priv.h
index 3477ee1..822b52b 100644
--- a/net/nimble/host/src/ble_l2cap_sig_priv.h
+++ b/net/nimble/host/src/ble_l2cap_sig_priv.h
@@ -80,6 +80,7 @@ int ble_l2cap_sig_reject_invalid_cid_tx(struct ble_hs_conn *conn,
                                         uint8_t id,
                                         uint16_t src_cid, uint16_t dst_cid);
 
+void ble_l2cap_sig_conn_broken(uint16_t conn_handle, int reason);
 int32_t ble_l2cap_sig_heartbeat(void);
 struct ble_l2cap_chan *ble_l2cap_sig_create_chan(void);
 int ble_l2cap_sig_init(void);

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/test/ble_gap_test.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/test/ble_gap_test.c b/net/nimble/host/src/test/ble_gap_test.c
index 5dd6532..2bbcfa9 100644
--- a/net/nimble/host/src/test/ble_gap_test.c
+++ b/net/nimble/host/src/test/ble_gap_test.c
@@ -41,16 +41,6 @@ static void *ble_gap_test_disc_arg;
  * $misc                                                                     *
  *****************************************************************************/
 
-static int
-ble_gap_test_util_update_in_progress(uint16_t conn_handle)
-{
-    ble_hs_conn_flags_t conn_flags;
-    int rc;
-
-    rc = ble_hs_atomic_conn_flags(conn_handle, &conn_flags);
-    return rc == 0 && conn_flags & BLE_HS_CONN_F_UPDATE;
-}
-
 static void
 ble_gap_test_util_reset_cb_info(void)
 {
@@ -344,7 +334,7 @@ ble_gap_test_util_verify_tx_params_reply_neg(uint8_t reason)
 static void
 ble_gap_test_util_rx_update_complete(
     uint8_t status,
-    struct ble_gap_upd_params *params)
+    const struct ble_gap_upd_params *params)
 {
     struct hci_le_conn_upd_complete evt;
 
@@ -1730,11 +1720,11 @@ TEST_SUITE(ble_gap_test_suite_stop_adv)
  *****************************************************************************/
 
 static void
-ble_gap_test_util_update(struct ble_gap_upd_params *params,
-                         int cmd_fail_idx, uint8_t hci_status,
-                         uint8_t event_status)
+ble_gap_test_util_update_no_l2cap(struct ble_gap_upd_params *params,
+                                  int master,
+                                  uint8_t hci_status, int event_status)
 {
-    int status;
+    struct ble_hs_conn *conn;
     int rc;
 
     uint8_t peer_addr[6] = { 1, 2, 3, 4, 5, 6 };
@@ -1744,6 +1734,19 @@ ble_gap_test_util_update(struct ble_gap_upd_params *params,
     ble_hs_test_util_create_conn(2, peer_addr, ble_gap_test_util_connect_cb,
                                  NULL);
 
+    if (!master) {
+        ble_hs_lock();
+        conn = ble_hs_conn_find(2);
+        TEST_ASSERT_FATAL(conn != NULL);
+        conn->bhc_flags &= ~BLE_HS_CONN_F_MASTER;
+        ble_hs_unlock();
+    }
+
+    /* Erase callback info reported during connection establishment; we only
+     * care about updates.
+     */
+    ble_gap_test_util_reset_cb_info();
+
     TEST_ASSERT(!ble_gap_master_in_progress());
 
     rc = ble_hs_test_util_conn_update(2, params, hci_status);
@@ -1754,49 +1757,269 @@ ble_gap_test_util_update(struct ble_gap_upd_params *params,
     ble_gap_test_util_verify_tx_update_conn(params);
 
     if (rc == 0) {
-        TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+        TEST_ASSERT(ble_gap_dbg_update_active(2));
+
+        /* Receive connection update complete event. */
+        ble_gap_test_util_rx_update_complete(event_status, params);
+
+        TEST_ASSERT(!ble_gap_master_in_progress());
+        TEST_ASSERT(!ble_gap_dbg_update_active(2));
+
+        TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
+        TEST_ASSERT(ble_gap_test_conn_status == BLE_HS_HCI_ERR(event_status));
+        if (event_status == 0) {
+            TEST_ASSERT(ble_gap_test_conn_desc.conn_handle == 2);
+            TEST_ASSERT(memcmp(ble_gap_test_conn_desc.peer_id_addr,
+                               peer_addr, 6) == 0);
+            TEST_ASSERT(ble_gap_test_conn_desc.conn_itvl == params->itvl_max);
+            TEST_ASSERT(ble_gap_test_conn_desc.conn_latency ==
+                        params->latency);
+            TEST_ASSERT(ble_gap_test_conn_desc.supervision_timeout ==
+                        params->supervision_timeout);
+        }
     } else {
-        TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
-        return;
+        TEST_ASSERT(!ble_gap_master_in_progress());
+        TEST_ASSERT(!ble_gap_dbg_update_active(2));
+
+        TEST_ASSERT(ble_gap_test_event.type == 0xff);
     }
+}
 
-    /* Receive connection update complete event. */
-    ble_gap_test_util_rx_update_complete(event_status, params);
+static void
+ble_gap_test_util_update_l2cap(struct ble_gap_upd_params *params,
+                               uint8_t hci_status, int event_status,
+                               uint16_t l2cap_result)
+{
+    struct ble_l2cap_sig_update_params l2cap_params;
+    struct ble_hs_conn *conn;
+    uint8_t id;
+    int rc;
+
+    uint8_t peer_addr[6] = { 1, 2, 3, 4, 5, 6 };
 
-    if (event_status != 0) {
-        status = BLE_HS_HCI_ERR(event_status);
-        goto fail;
+    ble_gap_test_util_init();
+
+    ble_hs_test_util_create_conn(2, peer_addr, ble_gap_test_util_connect_cb,
+                                 NULL);
+
+    ble_hs_lock();
+    conn = ble_hs_conn_find(2);
+    TEST_ASSERT_FATAL(conn != NULL);
+    conn->bhc_flags &= ~BLE_HS_CONN_F_MASTER;
+    ble_hs_unlock();
+
+    /* Erase callback info reported during connection establishment; we only
+     * care about updates.
+     */
+    ble_gap_test_util_reset_cb_info();
+
+    rc = ble_hs_test_util_conn_update(2, params, hci_status);
+    TEST_ASSERT(rc == 0);
+
+    /* Verify tx of connection update command. */
+    ble_gap_test_util_verify_tx_update_conn(params);
+
+    switch (hci_status) {
+    case 0:
+        /* Receive connection update complete event. */
+        ble_gap_test_util_rx_update_complete(event_status, params);
+        break;
+    case BLE_ERR_UNKNOWN_HCI_CMD:
+        break;
+    default:
+        TEST_ASSERT_FATAL(0);
+        break;
     }
 
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
+
+    l2cap_params.itvl_min = params->itvl_min;
+    l2cap_params.itvl_max = params->itvl_max;
+    l2cap_params.slave_latency = params->latency;
+    l2cap_params.timeout_multiplier = params->supervision_timeout;
+    id = ble_hs_test_util_verify_tx_l2cap_update_req(&l2cap_params);
+
+    /* Receive l2cap connection parameter update response. */
+    ble_hs_test_util_rx_l2cap_update_rsp(2, id, l2cap_result);
+    if (l2cap_result == BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT) {
+        TEST_ASSERT(ble_gap_dbg_update_active(2));
+
+        /* Receive connection update complete event. */
+        ble_gap_test_util_rx_update_complete(0, params);
+    }
+
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
+
+    TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
+    if (l2cap_result != BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT) {
+        TEST_ASSERT(ble_gap_test_conn_status == BLE_HS_EREJECT);
+    } else {
+        TEST_ASSERT(ble_gap_test_conn_status == 0);
+        TEST_ASSERT(ble_gap_test_conn_desc.conn_itvl == params->itvl_max);
+        TEST_ASSERT(ble_gap_test_conn_desc.conn_latency == params->latency);
+        TEST_ASSERT(ble_gap_test_conn_desc.supervision_timeout ==
+                    params->supervision_timeout);
+    }
+
+    TEST_ASSERT(ble_gap_test_conn_desc.conn_handle == 2);
+    TEST_ASSERT(memcmp(ble_gap_test_conn_desc.peer_id_addr,
+                       peer_addr, 6) == 0);
+}
+
+static void
+ble_gap_test_util_update_no_l2cap_tmo(struct ble_gap_upd_params *params,
+                                      int master)
+{
+    struct ble_hs_conn *conn;
+    int rc;
+
+    uint8_t peer_addr[6] = { 1, 2, 3, 4, 5, 6 };
+
+    ble_gap_test_util_init();
+
+    ble_hs_test_util_create_conn(2, peer_addr, ble_gap_test_util_connect_cb,
+                                 NULL);
+
+    if (!master) {
+        ble_hs_lock();
+        conn = ble_hs_conn_find(2);
+        TEST_ASSERT_FATAL(conn != NULL);
+        conn->bhc_flags &= ~BLE_HS_CONN_F_MASTER;
+        ble_hs_unlock();
+    }
+
+    /* Erase callback info reported during connection establishment; we only
+     * care about updates.
+     */
+    ble_gap_test_util_reset_cb_info();
+
+    TEST_ASSERT(!ble_gap_master_in_progress());
+
+    rc = ble_hs_test_util_conn_update(2, params, 0);
+    TEST_ASSERT(rc == 0);
     TEST_ASSERT(!ble_gap_master_in_progress());
 
+    /* Verify tx of connection update command. */
+    ble_gap_test_util_verify_tx_update_conn(params);
+
+    /* Ensure no update event reported. */
+    TEST_ASSERT(ble_gap_test_event.type == 0xff);
+
+    /* Advance 29 seconds; ensure no timeout reported. */
+    os_time_advance(29 * OS_TICKS_PER_SEC);
+    ble_gap_heartbeat();
+    TEST_ASSERT(ble_gap_test_event.type == 0xff);
+
+    /* Advance 30th second; ensure timeout reported. */
+    os_time_advance(1 * OS_TICKS_PER_SEC);
+
+    /* Timeout will result in a terminate HCI command being sent; schedule ack
+     * from controller.
+     */
+    ble_hs_test_util_set_ack_disconnect(0);
+
+    ble_gap_heartbeat();
+
+    /* Verify terminate was sent. */
+    ble_gap_test_util_verify_tx_disconnect();
+
     TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
-    TEST_ASSERT(ble_gap_test_conn_status == 0);
+    TEST_ASSERT(ble_gap_test_conn_status == BLE_HS_ETIMEOUT);
     TEST_ASSERT(ble_gap_test_conn_desc.conn_handle == 2);
     TEST_ASSERT(memcmp(ble_gap_test_conn_desc.peer_id_addr,
                        peer_addr, 6) == 0);
-    TEST_ASSERT(ble_gap_test_conn_desc.conn_itvl == params->itvl_max);
-    TEST_ASSERT(ble_gap_test_conn_desc.conn_latency == params->latency);
-    TEST_ASSERT(ble_gap_test_conn_desc.supervision_timeout ==
-        params->supervision_timeout);
+}
 
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+static void
+ble_gap_test_util_update_l2cap_tmo(struct ble_gap_upd_params *params,
+                                   uint8_t hci_status, uint8_t event_status,
+                                   int rx_l2cap)
+{
+    struct ble_l2cap_sig_update_params l2cap_params;
+    struct ble_hs_conn *conn;
+    uint8_t id;
+    int rc;
 
-    return;
+    uint8_t peer_addr[6] = { 1, 2, 3, 4, 5, 6 };
+
+    ble_gap_test_util_init();
+
+    ble_hs_test_util_create_conn(2, peer_addr, ble_gap_test_util_connect_cb,
+                                 NULL);
+
+    ble_hs_lock();
+    conn = ble_hs_conn_find(2);
+    TEST_ASSERT_FATAL(conn != NULL);
+    conn->bhc_flags &= ~BLE_HS_CONN_F_MASTER;
+    ble_hs_unlock();
+
+    /* Erase callback info reported during connection establishment; we only
+     * care about updates.
+     */
+    ble_gap_test_util_reset_cb_info();
+
+    rc = ble_hs_test_util_conn_update(2, params, hci_status);
+    TEST_ASSERT(rc == 0);
+
+    /* Verify tx of connection update command. */
+    ble_gap_test_util_verify_tx_update_conn(params);
+
+    switch (hci_status) {
+    case 0:
+        /* Receive connection update complete event. */
+        ble_gap_test_util_rx_update_complete(event_status, params);
+        break;
+    case BLE_ERR_UNKNOWN_HCI_CMD:
+        break;
+    default:
+        TEST_ASSERT_FATAL(0);
+        break;
+    }
+
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
+
+    if (rx_l2cap) {
+        l2cap_params.itvl_min = params->itvl_min;
+        l2cap_params.itvl_max = params->itvl_max;
+        l2cap_params.slave_latency = params->latency;
+        l2cap_params.timeout_multiplier = params->supervision_timeout;
+        id = ble_hs_test_util_verify_tx_l2cap_update_req(&l2cap_params);
+
+        /* Receive l2cap connection parameter update response. */
+        ble_hs_test_util_rx_l2cap_update_rsp(
+            2, id, BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT);
+    }
+
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
+
+    /* Ensure no update event reported. */
+    TEST_ASSERT(ble_gap_test_event.type == 0xff);
+
+    /* Advance 29 seconds; ensure no timeout reported. */
+    os_time_advance(29 * OS_TICKS_PER_SEC);
+    ble_gap_heartbeat();
+    ble_l2cap_sig_heartbeat();
+    TEST_ASSERT(ble_gap_test_event.type == 0xff);
+
+    /* Advance 30th second; ensure timeout reported. */
+    os_time_advance(1 * OS_TICKS_PER_SEC);
+
+    /* Timeout will result in a terminate HCI command being sent; schedule ack
+     * from controller.
+     */
+    ble_hs_test_util_set_ack_disconnect(0);
+
+    ble_gap_heartbeat();
+    ble_l2cap_sig_heartbeat();
+
+    /* Verify terminate was sent. */
+    ble_gap_test_util_verify_tx_disconnect();
 
-fail:
     TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
-    TEST_ASSERT(ble_gap_test_conn_status == status);
+    TEST_ASSERT(ble_gap_test_conn_status == BLE_HS_ETIMEOUT);
     TEST_ASSERT(ble_gap_test_conn_desc.conn_handle == 2);
     TEST_ASSERT(memcmp(ble_gap_test_conn_desc.peer_id_addr,
                        peer_addr, 6) == 0);
-    TEST_ASSERT(ble_gap_test_conn_desc.conn_itvl ==
-                BLE_GAP_INITIAL_CONN_ITVL_MAX);
-    TEST_ASSERT(ble_gap_test_conn_desc.conn_latency ==
-                BLE_GAP_INITIAL_CONN_LATENCY);
-    TEST_ASSERT(ble_gap_test_conn_desc.supervision_timeout ==
-                BLE_GAP_INITIAL_SUPERVISION_TIMEOUT);
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
 }
 
 static void
@@ -1830,7 +2053,7 @@ ble_gap_test_util_update_peer(uint8_t status,
                     params->supervision_timeout);
     }
 
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 }
 
 static void
@@ -1858,19 +2081,21 @@ ble_gap_test_util_update_req_pos(struct ble_gap_upd_params *peer_params,
         goto hci_fail;
     }
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+
+    /* We don't maintain an update entry when the peer initiates. */
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     /* Verify tx of connection parameters reply command. */
     ble_gap_test_util_verify_tx_params_reply_pos();
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     /* Receive connection update complete event. */
     ble_gap_test_util_rx_update_complete(0, self_params);
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
     TEST_ASSERT(ble_gap_test_conn_status == 0);
@@ -1916,7 +2141,7 @@ ble_gap_test_util_update_req_neg(struct ble_gap_upd_params *peer_params,
                                  &reason);
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     rc = ble_gap_test_util_rx_param_req(peer_params, 0, &cmd_idx, cmd_fail_idx,
                                         hci_status);
@@ -1924,13 +2149,13 @@ ble_gap_test_util_update_req_neg(struct ble_gap_upd_params *peer_params,
         goto hci_fail;
     }
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     /* Verify tx of connection parameters negative reply command. */
     ble_gap_test_util_verify_tx_params_reply_neg(reason);
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     return;
 
@@ -1968,7 +2193,7 @@ ble_gap_test_util_update_req_concurrent(
                                  NULL);
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     hci_status = cmd_fail_idx == 0 ? fail_status : 0;
     rc = ble_hs_test_util_conn_update(2, init_params, hci_status);
@@ -1980,14 +2205,14 @@ ble_gap_test_util_update_req_concurrent(
     ble_gap_test_util_verify_tx_update_conn(init_params);
 
     if (rc == 0) {
-        TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+        TEST_ASSERT(ble_gap_dbg_update_active(2));
     } else {
-        TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+        TEST_ASSERT(!ble_gap_dbg_update_active(2));
         return;
     }
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
 
     /* Receive connection parameter update request from peer. */
     ble_gap_test_conn_self_params = *self_params;
@@ -1997,19 +2222,19 @@ ble_gap_test_util_update_req_concurrent(
         goto hci_fail;
     }
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
 
     /* Verify tx of connection parameters reply command. */
     ble_gap_test_util_verify_tx_params_reply_pos();
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(ble_gap_dbg_update_active(2));
 
     /* Receive connection update complete event. */
     ble_gap_test_util_rx_update_complete(0, self_params);
 
     TEST_ASSERT(!ble_gap_master_in_progress());
-    TEST_ASSERT(!ble_gap_test_util_update_in_progress(2));
+    TEST_ASSERT(!ble_gap_dbg_update_active(2));
 
     TEST_ASSERT(ble_gap_test_event.type == BLE_GAP_EVENT_CONN_UPDATE);
     TEST_ASSERT(ble_gap_test_conn_status == 0);
@@ -2039,7 +2264,7 @@ hci_fail:
 
 TEST_CASE(ble_gap_test_case_update_conn_good)
 {
-    ble_gap_test_util_update(
+    ble_gap_test_util_update_no_l2cap(
         ((struct ble_gap_upd_params[]) { {
             .itvl_min = 10,
             .itvl_max = 100,
@@ -2047,9 +2272,9 @@ TEST_CASE(ble_gap_test_case_update_conn_good)
             .min_ce_len = 123,
             .max_ce_len = 456,
         }}),
-        -1, 0, 0);
+        1, 0, 0);
 
-    ble_gap_test_util_update(
+    ble_gap_test_util_update_no_l2cap(
         ((struct ble_gap_upd_params[]) { {
             .itvl_min = 100,
             .itvl_max = 100,
@@ -2057,12 +2282,12 @@ TEST_CASE(ble_gap_test_case_update_conn_good)
             .min_ce_len = 554,
             .max_ce_len = 554,
         }}),
-        -1, 0, 0);
+        1, 0, 0);
 }
 
 TEST_CASE(ble_gap_test_case_update_conn_bad)
 {
-    ble_gap_test_util_update(
+    ble_gap_test_util_update_no_l2cap(
         ((struct ble_gap_upd_params[]) { {
             .itvl_min = 10,
             .itvl_max = 100,
@@ -2070,12 +2295,12 @@ TEST_CASE(ble_gap_test_case_update_conn_bad)
             .min_ce_len = 123,
             .max_ce_len = 456,
         }}),
-        -1, 0, BLE_ERR_LMP_COLLISION);
+        1, 0, BLE_ERR_LMP_COLLISION);
 }
 
 TEST_CASE(ble_gap_test_case_update_conn_hci_fail)
 {
-    ble_gap_test_util_update(
+    ble_gap_test_util_update_no_l2cap(
         ((struct ble_gap_upd_params[]) { {
             .itvl_min = 10,
             .itvl_max = 100,
@@ -2083,7 +2308,39 @@ TEST_CASE(ble_gap_test_case_update_conn_hci_fail)
             .min_ce_len = 123,
             .max_ce_len = 456,
         }}),
-        0, BLE_ERR_UNSUPPORTED, 0);
+        1, BLE_ERR_UNSUPPORTED, 0);
+}
+
+TEST_CASE(ble_gap_test_case_update_conn_l2cap)
+{
+    struct ble_gap_upd_params params = {
+        .itvl_min = 10,
+        .itvl_max = 100,
+        .supervision_timeout = 0,
+        .min_ce_len = 123,
+        .max_ce_len = 456,
+    };
+
+    /* Accepted L2CAP failover; success. */
+    ble_gap_test_util_update_l2cap(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0,
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT);
+    ble_gap_test_util_update_l2cap(&params, 0, BLE_ERR_UNSUPP_REM_FEATURE,
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT);
+
+    /* Accepted L2CAP failover; failure. */
+    ble_gap_test_util_update_l2cap(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0, 
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT);
+    ble_gap_test_util_update_l2cap(&params, 0, BLE_ERR_UNSUPP_REM_FEATURE,
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_ACCEPT);
+
+    /* Rejected L2CAP failovers. */
+    ble_gap_test_util_update_l2cap(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0, 
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_REJECT);
+    ble_gap_test_util_update_l2cap(&params, 0, BLE_ERR_UNSUPP_REM_FEATURE,
+                                   BLE_L2CAP_SIG_UPDATE_RSP_RESULT_REJECT);
+
+    /* Don't attempt L2CAP failover on other event status. */
+    ble_gap_test_util_update_l2cap(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0, 0);
 }
 
 TEST_CASE(ble_gap_test_case_update_peer_good)
@@ -2142,7 +2399,6 @@ TEST_CASE(ble_gap_test_case_update_req_good)
             .max_ce_len = 554,
         }}),
         -1, 0);
-
 }
 
 TEST_CASE(ble_gap_test_case_update_req_hci_fail)
@@ -2297,6 +2553,7 @@ TEST_SUITE(ble_gap_test_suite_update_conn)
     ble_gap_test_case_update_conn_good();
     ble_gap_test_case_update_conn_bad();
     ble_gap_test_case_update_conn_hci_fail();
+    ble_gap_test_case_update_conn_l2cap();
     ble_gap_test_case_update_peer_good();
     ble_gap_test_case_update_req_good();
     ble_gap_test_case_update_req_hci_fail();
@@ -2454,6 +2711,34 @@ ble_gap_test_util_disc_timeout(int32_t duration_ms)
     ble_gap_test_util_reset_cb_info();
 }
 
+TEST_CASE(ble_gap_test_case_update_timeout)
+{
+    struct ble_gap_upd_params params = {
+        .itvl_min = 10,
+        .itvl_max = 100,
+        .supervision_timeout = 0,
+        .min_ce_len = 123,
+        .max_ce_len = 456,
+    };
+
+    /* No L2CAP. */
+    ble_gap_test_util_update_no_l2cap_tmo(&params, 1);
+
+    /* L2CAP - Local unsupported; L2CAP timeout. */
+    ble_gap_test_util_update_l2cap_tmo(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0, 0);
+
+    /* L2CAP - Local unsupported; LL timeout. */
+    ble_gap_test_util_update_l2cap_tmo(&params, BLE_ERR_UNKNOWN_HCI_CMD, 0, 1);
+
+    /* L2CAP - Remote unsupported; L2CAP timeout. */
+    ble_gap_test_util_update_l2cap_tmo(&params, 0, BLE_ERR_UNSUPP_REM_FEATURE,
+                                       0);
+
+    /* L2CAP - Remote unsupported; LL timeout. */
+    ble_gap_test_util_update_l2cap_tmo(&params, 0, BLE_ERR_UNSUPP_REM_FEATURE,
+                                       1);
+}
+
 TEST_CASE(ble_gap_test_case_conn_timeout_conn_forever)
 {
     ble_gap_test_util_init();
@@ -2557,6 +2842,8 @@ TEST_SUITE(ble_gap_test_suite_timeout)
     ble_gap_test_case_disc_forever_disc_timeout();
 
     ble_gap_test_case_conn_timeout_disc_timeout();
+
+    ble_gap_test_case_update_timeout();
 }
 
 TEST_CASE(ble_gap_test_case_mtu_us)

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/test/ble_hs_test_util.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/test/ble_hs_test_util.c b/net/nimble/host/src/test/ble_hs_test_util.c
index e1acb81..f30f65e 100644
--- a/net/nimble/host/src/test/ble_hs_test_util.c
+++ b/net/nimble/host/src/test/ble_hs_test_util.c
@@ -548,16 +548,21 @@ ble_hs_test_util_conn_cancel_full(void)
     TEST_ASSERT_FATAL(rc == 0);
 }
 
-int
-ble_hs_test_util_conn_terminate(uint16_t conn_handle, uint8_t hci_status)
+void
+ble_hs_test_util_set_ack_disconnect(uint8_t hci_status)
 {
-    int rc;
-
     ble_hs_test_util_set_ack(
         ble_hs_hci_util_opcode_join(BLE_HCI_OGF_LINK_CTRL,
                                     BLE_HCI_OCF_DISCONNECT_CMD),
         hci_status);
+}
 
+int
+ble_hs_test_util_conn_terminate(uint16_t conn_handle, uint8_t hci_status)
+{
+    int rc;
+
+    ble_hs_test_util_set_ack_disconnect(hci_status);
     rc = ble_gap_terminate(conn_handle, BLE_ERR_REM_USER_CONN_TERM);
     return rc;
 }
@@ -1208,6 +1213,105 @@ ble_hs_test_util_verify_tx_err_rsp(uint8_t req_op, uint16_t handle,
     TEST_ASSERT(rsp.baep_error_code == error_code);
 }
 
+static struct os_mbuf *
+ble_hs_test_util_verify_tx_l2cap_sig_hdr(uint8_t op, uint8_t id,
+                                   uint16_t payload_len,
+                                   struct ble_l2cap_sig_hdr *out_hdr)
+{
+    struct ble_l2cap_sig_hdr hdr;
+    struct os_mbuf *om;
+
+    om = ble_hs_test_util_prev_tx_dequeue();
+    TEST_ASSERT_FATAL(om != NULL);
+
+    TEST_ASSERT(OS_MBUF_PKTLEN(om) == BLE_L2CAP_SIG_HDR_SZ + payload_len);
+    ble_l2cap_sig_hdr_parse(om->om_data, om->om_len, &hdr);
+    TEST_ASSERT(hdr.op == op);
+    if (id != 0) {
+        TEST_ASSERT(hdr.identifier == id);
+    }
+    TEST_ASSERT(hdr.length == payload_len);
+
+    om->om_data += BLE_L2CAP_SIG_HDR_SZ;
+    om->om_len -= BLE_L2CAP_SIG_HDR_SZ;
+
+    if (out_hdr != NULL) {
+        *out_hdr = hdr;
+    }
+
+    return om;
+}
+
+/**
+ * @return                      The L2CAP sig identifier in the request.
+ */
+uint8_t
+ble_hs_test_util_verify_tx_l2cap_update_req(
+    struct ble_l2cap_sig_update_params *params)
+{
+    struct ble_l2cap_sig_update_req req;
+    struct ble_l2cap_sig_hdr hdr;
+    struct os_mbuf *om;
+
+    ble_hs_test_util_tx_all();
+
+    om = ble_hs_test_util_verify_tx_l2cap_sig_hdr(BLE_L2CAP_SIG_OP_UPDATE_REQ,
+                                                  0,
+                                                  BLE_L2CAP_SIG_UPDATE_REQ_SZ,
+                                                  &hdr);
+
+    /* Verify payload. */
+    ble_l2cap_sig_update_req_parse(om->om_data, om->om_len, &req);
+    TEST_ASSERT(req.itvl_min == params->itvl_min);
+    TEST_ASSERT(req.itvl_max == params->itvl_max);
+    TEST_ASSERT(req.slave_latency == params->slave_latency);
+    TEST_ASSERT(req.timeout_multiplier == params->timeout_multiplier);
+
+    return hdr.identifier;
+}
+
+int
+ble_hs_test_util_rx_l2cap_update_rsp(uint16_t conn_handle,
+                                     uint8_t id, uint16_t result)
+{
+    struct ble_l2cap_sig_update_rsp rsp;
+    struct hci_data_hdr hci_hdr;
+    struct os_mbuf *om;
+    void *v;
+    int rc;
+
+    hci_hdr = BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(
+        2, BLE_HCI_PB_FIRST_FLUSH,
+        BLE_L2CAP_HDR_SZ + BLE_L2CAP_SIG_HDR_SZ + BLE_L2CAP_SIG_UPDATE_RSP_SZ);
+
+    rc = ble_l2cap_sig_init_cmd(BLE_L2CAP_SIG_OP_UPDATE_RSP, id,
+                                BLE_L2CAP_SIG_UPDATE_RSP_SZ, &om, &v);
+    TEST_ASSERT_FATAL(rc == 0);
+
+    rsp.result = result;
+    ble_l2cap_sig_update_rsp_write(v, BLE_L2CAP_SIG_UPDATE_RSP_SZ, &rsp);
+
+    rc = ble_hs_test_util_l2cap_rx_first_frag(conn_handle, BLE_L2CAP_CID_SIG,
+                                              &hci_hdr, om);
+    return rc;
+}
+
+void
+ble_hs_test_util_verify_tx_l2cap_update_rsp(uint8_t exp_id,
+                                            uint16_t exp_result)
+{
+    struct ble_l2cap_sig_update_rsp rsp;
+    struct os_mbuf *om;
+
+    om = ble_hs_test_util_verify_tx_l2cap_sig_hdr(BLE_L2CAP_SIG_OP_UPDATE_RSP,
+                                            exp_id,
+                                            BLE_L2CAP_SIG_UPDATE_RSP_SZ,
+                                            NULL);
+
+    ble_l2cap_sig_update_rsp_parse(om->om_data, om->om_len, &rsp);
+    TEST_ASSERT(rsp.result == exp_result);
+}
+
 void
 ble_hs_test_util_set_static_rnd_addr(void)
 {

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/test/ble_hs_test_util.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/test/ble_hs_test_util.h b/net/nimble/host/src/test/ble_hs_test_util.h
index 7780d4e..0974a9a 100644
--- a/net/nimble/host/src/test/ble_hs_test_util.h
+++ b/net/nimble/host/src/test/ble_hs_test_util.h
@@ -50,6 +50,13 @@ struct ble_hs_test_util_mbuf_params {
     unsigned prep_list:1;
 };
 
+#define BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(handle, pb, len) \
+    ((struct hci_data_hdr) {                            \
+        .hdh_handle_pb_bc = ((handle)  << 0) |          \
+                            ((pb)      << 12),          \
+        .hdh_len = (len)                                \
+    })
+
 void ble_hs_test_util_prev_tx_enqueue(struct os_mbuf *om);
 struct os_mbuf *ble_hs_test_util_prev_tx_dequeue(void);
 struct os_mbuf *ble_hs_test_util_prev_tx_dequeue_pullup(void);
@@ -87,6 +94,7 @@ int ble_hs_test_util_connect(uint8_t own_addr_type,
                                    uint8_t ack_status);
 int ble_hs_test_util_conn_cancel(uint8_t ack_status);
 void ble_hs_test_util_conn_cancel_full(void);
+void ble_hs_test_util_set_ack_disconnect(uint8_t hci_status);
 int ble_hs_test_util_conn_terminate(uint16_t conn_handle, uint8_t hci_status);
 void ble_hs_test_util_conn_disconnect(uint16_t conn_handle);
 int ble_hs_test_util_exp_hci_status(int cmd_idx, int fail_idx,
@@ -148,6 +156,12 @@ void ble_hs_test_util_verify_tx_write_rsp(void);
 void ble_hs_test_util_verify_tx_mtu_cmd(int is_req, uint16_t mtu);
 void ble_hs_test_util_verify_tx_err_rsp(uint8_t req_op, uint16_t handle,
                                         uint8_t error_code);
+uint8_t ble_hs_test_util_verify_tx_l2cap_update_req(
+    struct ble_l2cap_sig_update_params *params);
+int ble_hs_test_util_rx_l2cap_update_rsp(uint16_t conn_handle,
+                                         uint8_t id, uint16_t result);
+void ble_hs_test_util_verify_tx_l2cap_update_rsp(uint8_t exp_id,
+                                                 uint16_t exp_result);
 void ble_hs_test_util_set_static_rnd_addr(void);
 struct os_mbuf *ble_hs_test_util_om_from_flat(const void *buf, uint16_t len);
 int ble_hs_test_util_flat_attr_cmp(const struct ble_hs_test_util_flat_attr *a,

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a3704587/net/nimble/host/src/test/ble_l2cap_test.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/test/ble_l2cap_test.c b/net/nimble/host/src/test/ble_l2cap_test.c
index 69db2f8..9410b3d 100644
--- a/net/nimble/host/src/test/ble_l2cap_test.c
+++ b/net/nimble/host/src/test/ble_l2cap_test.c
@@ -26,6 +26,7 @@
 
 #define BLE_L2CAP_TEST_CID  99
 
+static uint16_t ble_l2cap_test_update_conn_handle;
 static int ble_l2cap_test_update_status;
 static void *ble_l2cap_test_update_arg;
 
@@ -33,17 +34,11 @@ static void *ble_l2cap_test_update_arg;
  * $util                                                                     *
  *****************************************************************************/
 
-#define BLE_L2CAP_TEST_UTIL_HCI_HDR(handle, pb, len)    \
-    ((struct hci_data_hdr) {                            \
-        .hdh_handle_pb_bc = ((handle)  << 0) |          \
-                            ((pb)      << 12),          \
-        .hdh_len = (len)                                \
-    })
-
 static void
 ble_l2cap_test_util_init(void)
 {
     ble_hs_test_util_init();
+    ble_l2cap_test_update_conn_handle = BLE_HS_CONN_HANDLE_NONE;
     ble_l2cap_test_update_status = -1;
     ble_l2cap_test_update_arg = (void *)(uintptr_t)-1;
 }
@@ -58,7 +53,7 @@ ble_l2cap_test_util_rx_update_req(uint16_t conn_handle, uint8_t id,
     void *v;
     int rc;
 
-    hci_hdr = BLE_L2CAP_TEST_UTIL_HCI_HDR(
+    hci_hdr = BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(
         2, BLE_HCI_PB_FIRST_FLUSH,
         BLE_L2CAP_HDR_SZ + BLE_L2CAP_SIG_HDR_SZ + BLE_L2CAP_SIG_UPDATE_REQ_SZ);
 
@@ -80,102 +75,6 @@ ble_l2cap_test_util_rx_update_req(uint16_t conn_handle, uint8_t id,
     TEST_ASSERT_FATAL(rc == 0);
 }
 
-static int
-ble_l2cap_test_util_rx_update_rsp(uint16_t conn_handle,
-                                  uint8_t id, uint16_t result)
-{
-    struct ble_l2cap_sig_update_rsp rsp;
-    struct hci_data_hdr hci_hdr;
-    struct os_mbuf *om;
-    void *v;
-    int rc;
-
-    hci_hdr = BLE_L2CAP_TEST_UTIL_HCI_HDR(
-        2, BLE_HCI_PB_FIRST_FLUSH,
-        BLE_L2CAP_HDR_SZ + BLE_L2CAP_SIG_HDR_SZ + BLE_L2CAP_SIG_UPDATE_RSP_SZ);
-
-    rc = ble_l2cap_sig_init_cmd(BLE_L2CAP_SIG_OP_UPDATE_RSP, id,
-                                BLE_L2CAP_SIG_UPDATE_RSP_SZ, &om, &v);
-    TEST_ASSERT_FATAL(rc == 0);
-
-    rsp.result = result;
-    ble_l2cap_sig_update_rsp_write(v, BLE_L2CAP_SIG_UPDATE_RSP_SZ, &rsp);
-
-    rc = ble_hs_test_util_l2cap_rx_first_frag(conn_handle, BLE_L2CAP_CID_SIG,
-                                              &hci_hdr, om);
-    return rc;
-}
-
-
-static struct os_mbuf *
-ble_l2cap_test_util_verify_tx_sig_hdr(uint8_t op, uint8_t id,
-                                      uint16_t payload_len,
-                                      struct ble_l2cap_sig_hdr *out_hdr)
-{
-    struct ble_l2cap_sig_hdr hdr;
-    struct os_mbuf *om;
-
-    om = ble_hs_test_util_prev_tx_dequeue();
-    TEST_ASSERT_FATAL(om != NULL);
-
-    TEST_ASSERT(OS_MBUF_PKTLEN(om) == BLE_L2CAP_SIG_HDR_SZ + payload_len);
-    ble_l2cap_sig_hdr_parse(om->om_data, om->om_len, &hdr);
-    TEST_ASSERT(hdr.op == op);
-    if (id != 0) {
-        TEST_ASSERT(hdr.identifier == id);
-    }
-    TEST_ASSERT(hdr.length == payload_len);
-
-    om->om_data += BLE_L2CAP_SIG_HDR_SZ;
-    om->om_len -= BLE_L2CAP_SIG_HDR_SZ;
-
-    if (out_hdr != NULL) {
-        *out_hdr = hdr;
-    }
-
-    return om;
-}
-
-/**
- * @return                      The L2CAP sig identifier in the request.
- */
-static uint8_t
-ble_l2cap_test_util_verify_tx_update_req(
-    struct ble_l2cap_sig_update_params *params)
-{
-    struct ble_l2cap_sig_update_req req;
-    struct ble_l2cap_sig_hdr hdr;
-    struct os_mbuf *om;
-
-    om = ble_l2cap_test_util_verify_tx_sig_hdr(BLE_L2CAP_SIG_OP_UPDATE_REQ, 0,
-                                               BLE_L2CAP_SIG_UPDATE_REQ_SZ,
-                                               &hdr);
-
-    /* Verify payload. */
-    ble_l2cap_sig_update_req_parse(om->om_data, om->om_len, &req);
-    TEST_ASSERT(req.itvl_min == params->itvl_min);
-    TEST_ASSERT(req.itvl_max == params->itvl_max);
-    TEST_ASSERT(req.slave_latency == params->slave_latency);
-    TEST_ASSERT(req.timeout_multiplier == params->timeout_multiplier);
-
-    return hdr.identifier;
-}
-
-static void
-ble_l2cap_test_util_verify_tx_update_rsp(uint8_t exp_id, uint16_t exp_result)
-{
-    struct ble_l2cap_sig_update_rsp rsp;
-    struct os_mbuf *om;
-
-    om = ble_l2cap_test_util_verify_tx_sig_hdr(BLE_L2CAP_SIG_OP_UPDATE_RSP,
-                                               exp_id,
-                                               BLE_L2CAP_SIG_UPDATE_RSP_SZ,
-                                               NULL);
-
-    ble_l2cap_sig_update_rsp_parse(om->om_data, om->om_len, &rsp);
-    TEST_ASSERT(rsp.result == exp_result);
-}
-
 static void
 ble_l2cap_test_util_verify_tx_update_conn(
     struct ble_gap_upd_params *params)
@@ -252,7 +151,7 @@ ble_l2cap_test_util_rx_first_frag(uint16_t conn_handle,
     TEST_ASSERT_FATAL(om != NULL);
 
     hci_len = sizeof hci_hdr + l2cap_frag_len;
-    hci_hdr = BLE_L2CAP_TEST_UTIL_HCI_HDR(conn_handle,
+    hci_hdr = BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(conn_handle,
                                           BLE_HCI_PB_FIRST_FLUSH, hci_len);
     rc = ble_hs_test_util_l2cap_rx(conn_handle, &hci_hdr, om);
     return rc;
@@ -272,7 +171,7 @@ ble_l2cap_test_util_rx_next_frag(uint16_t conn_handle, uint16_t hci_len)
     v = os_mbuf_extend(om, hci_len);
     TEST_ASSERT_FATAL(v != NULL);
 
-    hci_hdr = BLE_L2CAP_TEST_UTIL_HCI_HDR(conn_handle,
+    hci_hdr = BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(conn_handle,
                                           BLE_HCI_PB_MIDDLE, hci_len);
     rc = ble_hs_test_util_l2cap_rx(conn_handle, &hci_hdr, om);
     return rc;
@@ -372,7 +271,7 @@ TEST_CASE(ble_l2cap_test_case_frag_single)
                                     NULL, NULL);
 
     /*** HCI header specifies middle fragment without start. */
-    hci_hdr = BLE_L2CAP_TEST_UTIL_HCI_HDR(2, BLE_HCI_PB_MIDDLE, 10);
+    hci_hdr = BLE_HS_TEST_UTIL_L2CAP_HCI_HDR(2, BLE_HCI_PB_MIDDLE, 10);
 
     om = ble_hs_mbuf_l2cap_pkt();
     TEST_ASSERT_FATAL(om != NULL);
@@ -467,7 +366,7 @@ TEST_CASE(ble_l2cap_test_case_sig_unsol_rsp)
                                     NULL, NULL);
 
     /* Receive an unsolicited response. */
-    rc = ble_l2cap_test_util_rx_update_rsp(2, 100, 0);
+    rc = ble_hs_test_util_rx_l2cap_update_rsp(2, 100, 0);
     TEST_ASSERT(rc == BLE_HS_ENOENT);
 
     /* Ensure we did not send anything in return. */
@@ -499,8 +398,6 @@ ble_l2cap_test_util_peer_updates(int accept)
 {
     struct ble_l2cap_sig_update_params l2cap_params;
     struct ble_gap_upd_params params;
-    ble_hs_conn_flags_t conn_flags;
-    int rc;
 
     ble_l2cap_test_util_init();
 
@@ -516,7 +413,7 @@ ble_l2cap_test_util_peer_updates(int accept)
 
     /* Ensure an update response command got sent. */
     ble_hs_process_tx_data_queue();
-    ble_l2cap_test_util_verify_tx_update_rsp(1, !accept);
+    ble_hs_test_util_verify_tx_l2cap_update_rsp(1, !accept);
 
     if (accept) {
         params.itvl_min = 0x200;
@@ -528,14 +425,14 @@ ble_l2cap_test_util_peer_updates(int accept)
         ble_l2cap_test_util_verify_tx_update_conn(&params);
     } else {
         /* Ensure no update got scheduled. */
-        rc = ble_hs_atomic_conn_flags(2, &conn_flags);
-        TEST_ASSERT(rc == 0 && !(conn_flags & BLE_HS_CONN_F_UPDATE));
+        TEST_ASSERT(!ble_gap_dbg_update_active(2));
     }
 }
 
 static void
-ble_l2cap_test_util_update_cb(int status, void *arg)
+ble_l2cap_test_util_update_cb(uint16_t conn_handle, int status, void *arg)
 {
+    ble_l2cap_test_update_conn_handle = conn_handle;
     ble_l2cap_test_update_status = status;
     ble_l2cap_test_update_arg = arg;
 }
@@ -565,10 +462,10 @@ ble_l2cap_test_util_we_update(int peer_accepts)
     ble_hs_test_util_tx_all();
 
     /* Ensure an update request got sent. */
-    id = ble_l2cap_test_util_verify_tx_update_req(&params);
+    id = ble_hs_test_util_verify_tx_l2cap_update_req(&params);
 
     /* Receive response from peer. */
-    rc = ble_l2cap_test_util_rx_update_rsp(2, id, !peer_accepts);
+    rc = ble_hs_test_util_rx_l2cap_update_rsp(2, id, !peer_accepts);
     TEST_ASSERT(rc == 0);
 
     /* Ensure callback got called. */
@@ -646,17 +543,17 @@ TEST_CASE(ble_l2cap_test_case_sig_update_init_fail_bad_id)
     ble_hs_test_util_tx_all();
 
     /* Ensure an update request got sent. */
-    id = ble_l2cap_test_util_verify_tx_update_req(&params);
+    id = ble_hs_test_util_verify_tx_l2cap_update_req(&params);
 
     /* Receive response from peer with incorrect ID. */
-    rc = ble_l2cap_test_util_rx_update_rsp(2, id + 1, 0);
+    rc = ble_hs_test_util_rx_l2cap_update_rsp(2, id + 1, 0);
     TEST_ASSERT(rc == BLE_HS_ENOENT);
 
     /* Ensure callback did not get called. */
     TEST_ASSERT(ble_l2cap_test_update_status == -1);
 
     /* Receive response from peer with correct ID. */
-    rc = ble_l2cap_test_util_rx_update_rsp(2, id, 0);
+    rc = ble_hs_test_util_rx_l2cap_update_rsp(2, id, 0);
     TEST_ASSERT(rc == 0);
 
     /* Ensure callback got called. */