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 2018/11/21 22:47:32 UTC

[mynewt-nimble] branch master updated (9f5a1d5 -> 15b1e43)

This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git.


    from 9f5a1d5  porting: Fix Makefile that lists non-existing file
     new 71f95c4  host: Divide startup into two stages
     new 6709c86  Allow the host to be stopped and started again
     new 4eec25a  Mynewt: Stop the host on system shutdown
     new 766fea5  apps/btshell: Allow enable / disable of host
     new 68a7f78  Combine ble_gap_preempt() and ble_gap_ensure_preempted()
     new 15b1e43  rename ble_hs_enabled --> ble_hs_is_enabled

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 apps/btshell/src/cmd.c                  |  66 ++++++++++
 apps/btshell/src/main.c                 |   7 +
 nimble/host/include/host/ble_hs.h       |  32 ++++-
 nimble/host/include/host/ble_hs_stop.h  |  70 ++++++++++
 nimble/host/include/host/ble_hs_test.h  |   1 +
 nimble/host/pkg.yml                     |   3 +
 nimble/host/src/ble_gap.c               |  61 ++++++---
 nimble/host/src/ble_hs.c                | 148 +++++++++++++++++-----
 nimble/host/src/ble_hs_priv.h           |   6 +
 nimble/host/src/ble_hs_shutdown.c       |  69 ++++++++++
 nimble/host/src/ble_hs_stop.c           | 218 ++++++++++++++++++++++++++++++++
 nimble/host/syscfg.yml                  |   7 +
 nimble/host/test/src/ble_hs_pvcy_test.c |   1 +
 nimble/host/test/src/ble_hs_stop_test.c | 200 +++++++++++++++++++++++++++++
 nimble/host/test/src/ble_hs_test.c      |   1 +
 nimble/host/test/src/ble_hs_test_util.c |  13 +-
 nimble/host/test/src/ble_hs_test_util.h |   1 +
 porting/nimble/Makefile.defs            |   1 +
 18 files changed, 850 insertions(+), 55 deletions(-)
 create mode 100644 nimble/host/include/host/ble_hs_stop.h
 create mode 100644 nimble/host/src/ble_hs_shutdown.c
 create mode 100644 nimble/host/src/ble_hs_stop.c
 create mode 100644 nimble/host/test/src/ble_hs_stop_test.c


[mynewt-nimble] 01/06: host: Divide startup into two stages

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 71f95c45b4d51dd954b40c684b6e1c914a497d96
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Wed Oct 3 17:34:56 2018 -0700

    host: Divide startup into two stages
    
    Prior to this commit: host startup was done in a single stage by an
    event on the default event queue.  This creates a problem if the
    application wishes to run the host in a different event queue (i.e., not
    the default task).  The typical init sequence looks like this:
    
    1. Sysinit (start event enqueued to default event queue).
    2. Application configures host, possible changing the host event queue.
    3. Host start event runs in the default task.
    
    The problem is that the start event in step three determines that the
    host task is the currently running task.  This is always the default
    task, which is incorrect if the user reconfigured the event queue.
    
    Now: host startup is divided into two stages.  The first stage simply
    enqueues a second event on the host event queue.  The startup sequence
    now looks like this:
    
    1. Sysinit (start event enqueued to default event queue).
    2. Application configures host, possible changing the host event queue.
    3. Stage 1 start event runs in the default task (stage 2 event enqueued
    to host task).
    4. Stage 2 start event runs in the host task.
---
 nimble/host/include/host/ble_hs.h |  8 ++++--
 nimble/host/src/ble_hs.c          | 55 ++++++++++++++++++++++++++++++++-------
 2 files changed, 51 insertions(+), 12 deletions(-)

diff --git a/nimble/host/include/host/ble_hs.h b/nimble/host/include/host/ble_hs.h
index 192fdef..82d7ff5 100644
--- a/nimble/host/include/host/ble_hs.h
+++ b/nimble/host/include/host/ble_hs.h
@@ -301,7 +301,9 @@ int ble_hs_synced(void);
  * commands.  This function must be called before any other host functionality
  * is used, but it must be called after both the host and controller are
  * initialized.  Typically, the host-parent-task calls this function at the top
- * of its task routine.
+ * of its task routine.  This function must only be called in the host parent
+ * task.  A safe alternative for starting the stack from any task is to call
+ * `ble_hs_sched_start()`.
  *
  * If the host fails to synchronize with the controller (if the controller is
  * not fully booted, for example), the host will attempt to resynchronize every
@@ -319,7 +321,9 @@ int ble_hs_start(void);
  * configure the host package in the meantime.
  *
  * If auto-start is disabled, the application should use this function to start
- * the BLE stack.
+ * the BLE stack.  This function can be called at any time as long as the host
+ * is stopped.  When the host successfully starts, the application is notified
+ * via the ble_hs_cfg.sync_cb callback.
  */
 void ble_hs_sched_start(void);
 
diff --git a/nimble/host/src/ble_hs.c b/nimble/host/src/ble_hs.c
index 1718c98..edfd88f 100644
--- a/nimble/host/src/ble_hs.c
+++ b/nimble/host/src/ble_hs.c
@@ -38,7 +38,8 @@
 static void ble_hs_event_rx_hci_ev(struct ble_npl_event *ev);
 static void ble_hs_event_tx_notify(struct ble_npl_event *ev);
 static void ble_hs_event_reset(struct ble_npl_event *ev);
-static void ble_hs_event_start(struct ble_npl_event *ev);
+static void ble_hs_event_start_stage1(struct ble_npl_event *ev);
+static void ble_hs_event_start_stage2(struct ble_npl_event *ev);
 static void ble_hs_timer_sched(int32_t ticks_from_now);
 
 struct os_mempool ble_hs_hci_ev_pool;
@@ -52,7 +53,8 @@ static struct ble_npl_event ble_hs_ev_tx_notifications;
 /** OS event - triggers a full reset. */
 static struct ble_npl_event ble_hs_ev_reset;
 
-static struct ble_npl_event ble_hs_ev_start;
+static struct ble_npl_event ble_hs_ev_start_stage1;
+static struct ble_npl_event ble_hs_ev_start_stage2;
 
 uint8_t ble_hs_sync_state;
 static int ble_hs_reset_reason;
@@ -455,15 +457,22 @@ ble_hs_timer_resched(void)
      */
     ble_hs_timer_reset(0);
 }
- 
+
+static void
+ble_hs_sched_start_stage2(void)
+{
+    ble_npl_eventq_put((struct ble_npl_eventq *)ble_hs_evq_get(),
+                       &ble_hs_ev_start_stage2);
+}
+
 void
 ble_hs_sched_start(void)
 {
 #ifdef MYNEWT
     ble_npl_eventq_put((struct ble_npl_eventq *)os_eventq_dflt_get(),
-                       &ble_hs_ev_start);
+                       &ble_hs_ev_start_stage1);
 #else
-    ble_npl_eventq_put(nimble_port_get_dflt_eventq(), &ble_hs_ev_start);
+    ble_npl_eventq_put(nimble_port_get_dflt_eventq(), &ble_hs_ev_start_stage1);
 #endif
 }
 
@@ -504,8 +513,30 @@ ble_hs_event_reset(struct ble_npl_event *ev)
     ble_hs_reset();
 }
 
+/**
+ * Implements the first half of the start process.  This just enqueues another
+ * event on the host parent task's event queue.
+ *
+ * Starting is done in two stages to allow the application time to configure
+ * the event queue to use after system initialization but before the host
+ * starts.
+ */
+static void
+ble_hs_event_start_stage1(struct ble_npl_event *ev)
+{
+    ble_hs_sched_start_stage2();
+}
+
+/**
+ * Implements the second half of the start process.  This actually starts the
+ * host.
+ *
+ * Starting is done in two stages to allow the application time to configure
+ * the event queue to use after system initialization but before the host
+ * starts.
+ */
 static void
-ble_hs_event_start(struct ble_npl_event *ev)
+ble_hs_event_start_stage2(struct ble_npl_event *ev)
 {
     int rc;
 
@@ -651,9 +682,13 @@ ble_hs_init(void)
      */
     ble_hs_reset_reason = 0;
 
-    ble_npl_event_init(&ble_hs_ev_tx_notifications, ble_hs_event_tx_notify, NULL);
+    ble_npl_event_init(&ble_hs_ev_tx_notifications, ble_hs_event_tx_notify,
+                       NULL);
     ble_npl_event_init(&ble_hs_ev_reset, ble_hs_event_reset, NULL);
-    ble_npl_event_init(&ble_hs_ev_start, ble_hs_event_start, NULL);
+    ble_npl_event_init(&ble_hs_ev_start_stage1, ble_hs_event_start_stage1,
+                       NULL);
+    ble_npl_event_init(&ble_hs_ev_start_stage2, ble_hs_event_start_stage2,
+                       NULL);
 
 #if BLE_MONITOR
     rc = ble_monitor_init();
@@ -712,9 +747,9 @@ ble_hs_init(void)
 #if MYNEWT_VAL(BLE_HS_AUTO_START)
 #ifdef MYNEWT
     ble_npl_eventq_put((struct ble_npl_eventq *)os_eventq_dflt_get(),
-                       &ble_hs_ev_start);
+                       &ble_hs_ev_start_stage1);
 #else
-    ble_npl_eventq_put(nimble_port_get_dflt_eventq(), &ble_hs_ev_start);
+    ble_npl_eventq_put(nimble_port_get_dflt_eventq(), &ble_hs_ev_start_stage1);
 #endif
 #endif
 


[mynewt-nimble] 04/06: apps/btshell: Allow enable / disable of host

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 766fea564c2e3ac1602705546bb1656b3e7e5c18
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Thu Oct 4 12:28:20 2018 -0700

    apps/btshell: Allow enable / disable of host
    
    This commit adds two commands to btshell:
        * host-enable
        * host-disable
---
 apps/btshell/src/cmd.c  | 66 +++++++++++++++++++++++++++++++++++++++++++++++++
 apps/btshell/src/main.c |  7 ++++++
 2 files changed, 73 insertions(+)

diff --git a/apps/btshell/src/cmd.c b/apps/btshell/src/cmd.c
index 0ad0e8a..42507fd 100644
--- a/apps/btshell/src/cmd.c
+++ b/apps/btshell/src/cmd.c
@@ -3155,6 +3155,58 @@ static const struct shell_cmd_help phy_read_help = {
 };
 
 /*****************************************************************************
+ * $host-enable                                                              *
+ *****************************************************************************/
+
+static int
+cmd_host_enable(int argc, char **argv)
+{
+    ble_hs_sched_start();
+
+    return 0;
+}
+
+#if MYNEWT_VAL(SHELL_CMD_HELP)
+static const struct shell_cmd_help host_enable_help = {
+    .summary = "start the NimBLE host",
+    .usage = NULL,
+    .params = NULL,
+};
+#endif
+
+/*****************************************************************************
+ * $host-disable                                                              *
+ *****************************************************************************/
+
+static void
+on_stop(int status, void *arg)
+{
+    if (status == 0) {
+        console_printf("host stopped\n");
+    } else {
+        console_printf("host failed to stop; rc=%d\n", status);
+    }
+}
+
+static int
+cmd_host_disable(int argc, char **argv)
+{
+    static struct ble_hs_stop_listener listener;
+    int rc;
+
+    rc = ble_hs_stop(&listener, on_stop, NULL);
+    return rc;
+}
+
+#if MYNEWT_VAL(SHELL_CMD_HELP)
+static const struct shell_cmd_help host_disable_help = {
+    .summary = "stop the NimBLE host",
+    .usage = NULL,
+    .params = NULL,
+};
+#endif
+
+/*****************************************************************************
  * $gatt-discover                                                            *
  *****************************************************************************/
 
@@ -3856,6 +3908,20 @@ static const struct shell_cmd btshell_commands[] = {
         .help = &phy_read_help,
 #endif
     },
+    {
+        .sc_cmd = "host-enable",
+        .sc_cmd_func = cmd_host_enable,
+#if MYNEWT_VAL(SHELL_CMD_HELP)
+        .help = &host_enable_help,
+#endif
+    },
+    {
+        .sc_cmd = "host-disable",
+        .sc_cmd_func = cmd_host_disable,
+#if MYNEWT_VAL(SHELL_CMD_HELP)
+        .help = &host_disable_help,
+#endif
+    },
     { NULL, NULL, NULL },
 };
 
diff --git a/apps/btshell/src/main.c b/apps/btshell/src/main.c
index ab60527..f87efb9 100644
--- a/apps/btshell/src/main.c
+++ b/apps/btshell/src/main.c
@@ -1862,6 +1862,12 @@ btshell_on_reset(int reason)
     console_printf("Error: Resetting state; reason=%d\n", reason);
 }
 
+static void
+btshell_on_sync(void)
+{
+    console_printf("Host and controller synced\n");
+}
+
 #if MYNEWT_VAL(BLE_L2CAP_COC_MAX_NUM) != 0
 
 static int
@@ -2214,6 +2220,7 @@ main(int argc, char **argv)
 
     /* Initialize the NimBLE host configuration. */
     ble_hs_cfg.reset_cb = btshell_on_reset;
+    ble_hs_cfg.sync_cb = btshell_on_sync;
     ble_hs_cfg.gatts_register_cb = gatt_svr_register_cb;
     ble_hs_cfg.store_status_cb = ble_store_util_status_rr;
 


[mynewt-nimble] 03/06: Mynewt: Stop the host on system shutdown

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 4eec25a1c2862e9b96a73c144824df86f4936c94
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Wed Oct 3 17:57:15 2018 -0700

    Mynewt: Stop the host on system shutdown
    
    This is Mynewt-only feature.  When the system is shutting down, the host
    stops.
---
 nimble/host/include/host/ble_hs.h | 11 +++++++
 nimble/host/pkg.yml               |  3 ++
 nimble/host/src/ble_hs_shutdown.c | 69 +++++++++++++++++++++++++++++++++++++++
 nimble/host/syscfg.yml            |  7 ++++
 4 files changed, 90 insertions(+)

diff --git a/nimble/host/include/host/ble_hs.h b/nimble/host/include/host/ble_hs.h
index ba874b1..6719a6f 100644
--- a/nimble/host/include/host/ble_hs.h
+++ b/nimble/host/include/host/ble_hs.h
@@ -364,6 +364,17 @@ void ble_hs_evq_set(struct ble_npl_eventq *evq);
  */
 void ble_hs_init(void);
 
+/**
+ * @brief Called when the system is shutting down.  Stops the BLE host.
+ *
+ * @param reason                The reason for the shutdown.  One of the
+ *                                  HAL_RESET_[...] codes or an
+ *                                  implementation-defined value.
+ *
+ * @return                      SYSDOWN_IN_PROGRESS. 
+ */
+int ble_hs_shutdown(int reason);
+
 #ifdef __cplusplus
 }
 #endif
diff --git a/nimble/host/pkg.yml b/nimble/host/pkg.yml
index d91454e..7c8c021 100644
--- a/nimble/host/pkg.yml
+++ b/nimble/host/pkg.yml
@@ -50,3 +50,6 @@ pkg.req_apis:
 
 pkg.init:
     ble_hs_init: 200
+
+pkg.down.BLE_HS_STOP_ON_SHUTDOWN:
+    ble_hs_shutdown: 200
diff --git a/nimble/host/src/ble_hs_shutdown.c b/nimble/host/src/ble_hs_shutdown.c
new file mode 100644
index 0000000..f29d4a6
--- /dev/null
+++ b/nimble/host/src/ble_hs_shutdown.c
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#if MYNEWT
+
+#include "os/mynewt.h"
+#include "ble_hs_priv.h"
+
+static struct ble_hs_stop_listener ble_hs_shutdown_stop_listener;
+
+/**
+ * Called when the host stop procedure has completed.
+ */
+static void
+ble_hs_shutdown_stop_cb(int status, void *arg)
+{
+    SYSDOWN_ASSERT_ACTIVE();
+
+    /* Indicate to sysdown that the host is fully shut down. */
+    sysdown_release();
+}
+
+int
+ble_hs_shutdown(int reason)
+{
+    int rc;
+
+    /* Ensure this function only gets called by sysdown. */
+    SYSDOWN_ASSERT_ACTIVE();
+
+    /* Initiate a host stop procedure. */
+    rc = ble_hs_stop(&ble_hs_shutdown_stop_listener, ble_hs_shutdown_stop_cb,
+                     NULL);
+    switch (rc) {
+    case 0:
+        /* Stop initiated.  Wait for result to be reported asynchronously. */
+        return SYSDOWN_IN_PROGRESS;
+
+    case BLE_HS_EBUSY:
+        /* Already stopping.  Wait for result to be reported asynchronously. */
+        return SYSDOWN_IN_PROGRESS;
+
+    case BLE_HS_EALREADY:
+        /* Already stopped.  Shutdown complete. */
+        return SYSDOWN_COMPLETE;
+
+    default:
+        BLE_HS_LOG(ERROR, "ble_hs_shutdown: failed to stop host; rc=%d\n", rc);
+        return SYSDOWN_COMPLETE;
+    }
+}
+
+#endif
diff --git a/nimble/host/syscfg.yml b/nimble/host/syscfg.yml
index 90a61c1..d16d93f 100644
--- a/nimble/host/syscfg.yml
+++ b/nimble/host/syscfg.yml
@@ -422,5 +422,12 @@ syscfg.defs:
             a necessary workaround when interfacing with some controllers.
         value: 0
 
+    BLE_HS_STOP_ON_SHUTDOWN:
+        description: >
+            Stops the Bluetooth host when the system shuts down.  Stopping
+            entails aborting all GAP procedures and terminating open
+            connections.
+        value: 1
+
 syscfg.vals.BLE_MESH:
     BLE_SM_SC: 1


[mynewt-nimble] 06/06: rename ble_hs_enabled --> ble_hs_is_enabled

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 15b1e4345ce2f113122551c820d25bb1be14f29d
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Mon Nov 19 10:44:51 2018 -0800

    rename ble_hs_enabled --> ble_hs_is_enabled
---
 nimble/host/include/host/ble_hs.h | 2 +-
 nimble/host/src/ble_gap.c         | 8 ++++----
 nimble/host/src/ble_hs.c          | 4 ++--
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/nimble/host/include/host/ble_hs.h b/nimble/host/include/host/ble_hs.h
index 6719a6f..3bf4665 100644
--- a/nimble/host/include/host/ble_hs.h
+++ b/nimble/host/include/host/ble_hs.h
@@ -296,7 +296,7 @@ extern struct ble_hs_cfg ble_hs_cfg;
  * @return 1 if the host is enabled;
  *         0 if the host is disabled.
  */
-int ble_hs_enabled(void);
+int ble_hs_is_enabled(void);
 
 /**
  * Indicates whether the host has synchronized with the controller.
diff --git a/nimble/host/src/ble_gap.c b/nimble/host/src/ble_gap.c
index 6824f79..50e1596 100644
--- a/nimble/host/src/ble_gap.c
+++ b/nimble/host/src/ble_gap.c
@@ -2093,7 +2093,7 @@ ble_gap_adv_start(uint8_t own_addr_type, const ble_addr_t *direct_addr,
         }
     }
 
-    if (!ble_hs_enabled()) {
+    if (!ble_hs_is_enabled()) {
         rc = BLE_HS_EDISABLED;
         goto done;
     }
@@ -3104,7 +3104,7 @@ ble_gap_disc_ext_validate(uint8_t own_addr_type)
         return BLE_HS_EALREADY;
     }
 
-    if (!ble_hs_enabled()) {
+    if (!ble_hs_is_enabled()) {
         return BLE_HS_EDISABLED;
     }
 
@@ -3588,7 +3588,7 @@ ble_gap_ext_connect(uint8_t own_addr_type, const ble_addr_t *peer_addr,
         goto done;
     }
 
-    if (!ble_hs_enabled()) {
+    if (!ble_hs_is_enabled()) {
         return BLE_HS_EDISABLED;
     }
 
@@ -3715,7 +3715,7 @@ ble_gap_connect(uint8_t own_addr_type, const ble_addr_t *peer_addr,
         goto done;
     }
 
-    if (!ble_hs_enabled()) {
+    if (!ble_hs_is_enabled()) {
         rc = BLE_HS_EDISABLED;
         goto done;
     }
diff --git a/nimble/host/src/ble_hs.c b/nimble/host/src/ble_hs.c
index 3ee1757..2f50409 100644
--- a/nimble/host/src/ble_hs.c
+++ b/nimble/host/src/ble_hs.c
@@ -308,7 +308,7 @@ ble_hs_clear_rx_queue(void)
 }
 
 int
-ble_hs_enabled(void)
+ble_hs_is_enabled(void)
 {
     return ble_hs_enabled_state == BLE_HS_ENABLED_STATE_ON;
 }
@@ -442,7 +442,7 @@ ble_hs_timer_reset(uint32_t ticks)
 {
     int rc;
 
-    if (!ble_hs_enabled()) {
+    if (!ble_hs_is_enabled()) {
         ble_npl_callout_stop(&ble_hs_timer);
     } else {
         rc = ble_npl_callout_reset(&ble_hs_timer, ticks);


[mynewt-nimble] 05/06: Combine ble_gap_preempt() and ble_gap_ensure_preempted()

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 68a7f7815b11634ee2d3f6367a869ccb33f2d893
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Mon Nov 19 10:37:57 2018 -0800

    Combine ble_gap_preempt() and ble_gap_ensure_preempted()
---
 nimble/host/src/ble_gap.c      | 16 ++--------------
 nimble/host/src/ble_gap_priv.h |  1 -
 nimble/host/src/ble_hs_stop.c  |  2 +-
 3 files changed, 3 insertions(+), 16 deletions(-)

diff --git a/nimble/host/src/ble_gap.c b/nimble/host/src/ble_gap.c
index e131a7d..6824f79 100644
--- a/nimble/host/src/ble_gap.c
+++ b/nimble/host/src/ble_gap.c
@@ -4673,6 +4673,8 @@ ble_gap_preempt_no_lock(void)
 }
 
 /**
+ * @brief Preempts the GAP if it is not already preempted.
+ *
  * Aborts all active GAP procedures and prevents new ones from being started.
  * This function is used to ensure an idle GAP so that the controller's
  * resolving list can be modified.  When done accessing the resolving list, the
@@ -4687,20 +4689,6 @@ ble_gap_preempt(void)
 {
     ble_hs_lock();
 
-    BLE_HS_DBG_ASSERT(!ble_gap_is_preempted());
-    ble_gap_preempt_no_lock();
-
-    ble_hs_unlock();
-}
-
-/**
- * Preempts the GAP if it is not already preempted.
- */
-void
-ble_gap_ensure_preempted(void)
-{
-    ble_hs_lock();
-
     if (!ble_gap_is_preempted()) {
         ble_gap_preempt_no_lock();
     }
diff --git a/nimble/host/src/ble_gap_priv.h b/nimble/host/src/ble_gap_priv.h
index 0a566c6..a789f39 100644
--- a/nimble/host/src/ble_gap_priv.h
+++ b/nimble/host/src/ble_gap_priv.h
@@ -110,7 +110,6 @@ int ble_gap_repeat_pairing_event(const struct ble_gap_repeat_pairing *rp);
 int ble_gap_master_in_progress(void);
 
 void ble_gap_preempt(void);
-void ble_gap_ensure_preempted(void);
 void ble_gap_preempt_done(void);
 
 void ble_gap_conn_broken(uint16_t conn_handle, int reason);
diff --git a/nimble/host/src/ble_hs_stop.c b/nimble/host/src/ble_hs_stop.c
index 2c46188..31480d7 100644
--- a/nimble/host/src/ble_hs_stop.c
+++ b/nimble/host/src/ble_hs_stop.c
@@ -193,7 +193,7 @@ ble_hs_stop(struct ble_hs_stop_listener *listener,
     }
 
     /* Abort all active GAP procedures. */
-    ble_gap_ensure_preempted();
+    ble_gap_preempt();
     ble_gap_preempt_done();
 
     rc = ble_gap_event_listener_register(&ble_hs_stop_gap_listener,


[mynewt-nimble] 02/06: Allow the host to be stopped and started again

Posted by cc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ccollins pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mynewt-nimble.git

commit 6709c868f22bb3f7349cfd60995c468c4607b313
Author: Christopher Collins <cc...@apache.org>
AuthorDate: Wed Oct 3 17:57:02 2018 -0700

    Allow the host to be stopped and started again
    
    The application can stop the host by calling `ble_hs_stop()`.  Stopping
    the host has the following effects:
    * Abort all active GAP procedures with status BLE_HS_EPREEMPTED.
    * Disallow any subsequent GAP procedures until the host is started
      again.
    * Terminate all established connections.
    
    If multiple clients attempt to stop the host simultaneously, all of
    their provided callbacks are executed when the host stops.  The purpose
    of this behavior is to guarantee a consistent shutdown procedure.
---
 nimble/host/include/host/ble_hs.h       |  13 +-
 nimble/host/include/host/ble_hs_stop.h  |  70 ++++++++++
 nimble/host/include/host/ble_hs_test.h  |   1 +
 nimble/host/src/ble_gap.c               |  73 ++++++++---
 nimble/host/src/ble_gap_priv.h          |   1 +
 nimble/host/src/ble_hs.c                |  93 ++++++++++----
 nimble/host/src/ble_hs_priv.h           |   6 +
 nimble/host/src/ble_hs_stop.c           | 218 ++++++++++++++++++++++++++++++++
 nimble/host/test/src/ble_hs_pvcy_test.c |   1 +
 nimble/host/test/src/ble_hs_stop_test.c | 200 +++++++++++++++++++++++++++++
 nimble/host/test/src/ble_hs_test.c      |   1 +
 nimble/host/test/src/ble_hs_test_util.c |  13 +-
 nimble/host/test/src/ble_hs_test_util.h |   1 +
 porting/nimble/Makefile.defs            |   1 +
 14 files changed, 649 insertions(+), 43 deletions(-)

diff --git a/nimble/host/include/host/ble_hs.h b/nimble/host/include/host/ble_hs.h
index 82d7ff5..ba874b1 100644
--- a/nimble/host/include/host/ble_hs.h
+++ b/nimble/host/include/host/ble_hs.h
@@ -38,6 +38,7 @@
 #include "host/ble_hs_log.h"
 #include "host/ble_hs_test.h"
 #include "host/ble_hs_mbuf.h"
+#include "host/ble_hs_stop.h"
 #include "host/ble_ibeacon.h"
 #include "host/ble_l2cap.h"
 #include "host/ble_sm.h"
@@ -93,6 +94,7 @@ extern "C" {
 #define BLE_HS_ESTORE_CAP           27
 #define BLE_HS_ESTORE_FAIL          28
 #define BLE_HS_EPREEMPTED           29
+#define BLE_HS_EDISABLED            30
 
 /** Error base for ATT errors */
 #define BLE_HS_ERR_ATT_BASE         0x100
@@ -288,11 +290,20 @@ extern struct ble_hs_cfg ble_hs_cfg;
  */
 
 /**
+ * @brief Indicates whether the host is enabled.  The host is enabled if it is
+ * starting or fully started.  It is disabled if it is stopping or stopped.
+ *
+ * @return 1 if the host is enabled;
+ *         0 if the host is disabled.
+ */
+int ble_hs_enabled(void);
+
+/**
  * Indicates whether the host has synchronized with the controller.
  * Synchronization must occur before any host procedures can be performed.
  *
  * @return 1 if the host and controller are in sync;
- *         0 if the host and controller our out of sync.
+ *         0 if the host and controller are out of sync.
  */
 int ble_hs_synced(void);
 
diff --git a/nimble/host/include/host/ble_hs_stop.h b/nimble/host/include/host/ble_hs_stop.h
new file mode 100644
index 0000000..d16c9c2
--- /dev/null
+++ b/nimble/host/include/host/ble_hs_stop.h
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef H_BLE_HS_STOP_
+#define H_BLE_HS_STOP_
+
+/** @typedef ble_hs_stop_fn
+ * @brief Callback function; reports the result of a host stop procedure.
+ *
+ * @param status                The result of the host stop procedure.  One of
+ *                                  the HAL_RESET_[...] codes or an
+ *                                  implementation-defined value.
+ * @param arg                   Optional argument specified when the stop
+ *                                  procedure was initiated.
+ *
+ */
+typedef void ble_hs_stop_fn(int status, void *arg);
+
+/**
+ * @brief Used to report the result of a stop procedure.
+ *
+ * This should be used as an opaque structure and not modified manually.
+ */
+struct ble_hs_stop_listener {
+    ble_hs_stop_fn *fn;
+    void *arg;
+    SLIST_ENTRY(ble_hs_stop_listener) link;
+};
+
+/**
+ * @brief Stops the BLE host.
+ *
+ * Aborts all active GAP procedures and terminates all open connections.
+ * Connection termination is performed asynchronously, so this function's
+ * result is reported via the provided listener.
+ *
+ * @param listener              A listener to populate.  This object's initial
+ *                                  value doesn't matter, but its lifetime must
+ *                                  extend until the stop procedure completes.
+ * @param fn                    The callback to execute when the stop procedure
+ *                                  completes.
+ * @param arg                   Optional argument to pass to the callback.
+ *
+ * @return                      0: Stop procedure successfully initiated.
+ *                              BLE_HS_EBUSY: Stop procedure already in
+ *                                  progress; the provided callback gets called
+ *                                  when the procedure completes.
+ *                              BLE_HS_EALREADY: Host already stopped; the
+ *                                  provided callback does *not* get called.
+ */
+int ble_hs_stop(struct ble_hs_stop_listener *listener, 
+                ble_hs_stop_fn *fn, void *arg);
+
+#endif
diff --git a/nimble/host/include/host/ble_hs_test.h b/nimble/host/include/host/ble_hs_test.h
index 8fc580d..feb5104 100644
--- a/nimble/host/include/host/ble_hs_test.h
+++ b/nimble/host/include/host/ble_hs_test.h
@@ -50,6 +50,7 @@ int ble_hs_pvcy_test_all(void);
 int ble_sm_lgcy_test_suite(void);
 int ble_sm_sc_test_suite(void);
 int ble_sm_test_all(void);
+int ble_stop_test_all(void);
 int ble_store_test_all(void);
 int ble_uuid_test_all(void);
 
diff --git a/nimble/host/src/ble_gap.c b/nimble/host/src/ble_gap.c
index b931b66..e131a7d 100644
--- a/nimble/host/src/ble_gap.c
+++ b/nimble/host/src/ble_gap.c
@@ -2093,6 +2093,11 @@ ble_gap_adv_start(uint8_t own_addr_type, const ble_addr_t *direct_addr,
         }
     }
 
+    if (!ble_hs_enabled()) {
+        rc = BLE_HS_EDISABLED;
+        goto done;
+    }
+
     if (ble_gap_is_preempted()) {
         rc = BLE_HS_EPREEMPTED;
         goto done;
@@ -3099,6 +3104,10 @@ ble_gap_disc_ext_validate(uint8_t own_addr_type)
         return BLE_HS_EALREADY;
     }
 
+    if (!ble_hs_enabled()) {
+        return BLE_HS_EDISABLED;
+    }
+
     if (ble_gap_is_preempted()) {
         return BLE_HS_EPREEMPTED;
     }
@@ -3579,6 +3588,10 @@ ble_gap_ext_connect(uint8_t own_addr_type, const ble_addr_t *peer_addr,
         goto done;
     }
 
+    if (!ble_hs_enabled()) {
+        return BLE_HS_EDISABLED;
+    }
+
     if (ble_gap_is_preempted()) {
         rc = BLE_HS_EPREEMPTED;
         goto done;
@@ -3702,6 +3715,11 @@ ble_gap_connect(uint8_t own_addr_type, const ble_addr_t *peer_addr,
         goto done;
     }
 
+    if (!ble_hs_enabled()) {
+        rc = BLE_HS_EDISABLED;
+        goto done;
+    }
+
     if (ble_gap_is_preempted()) {
         rc = BLE_HS_EPREEMPTED;
         goto done;
@@ -3796,6 +3814,7 @@ ble_gap_conn_active(void)
 /*****************************************************************************
  * $terminate connection procedure                                           *
  *****************************************************************************/
+
 int
 ble_gap_terminate(uint16_t conn_handle, uint8_t hci_reason)
 {
@@ -4613,28 +4632,14 @@ ble_gap_mtu_event(uint16_t conn_handle, uint16_t cid, uint16_t mtu)
  * $preempt                                                                  *
  *****************************************************************************/
 
-/**
- * Aborts all active GAP procedures and prevents new ones from being started.
- * This function is used to ensure an idle GAP so that the controller's
- * resolving list can be modified.  When done accessing the resolving list, the
- * caller must call `ble_gap_preempt_done()` to permit new GAP procedures.
- *
- * On preemption, all aborted GAP procedures are reported with a status or
- * reason code of BLE_HS_EPREEMPTED.  An attempt to initiate a new GAP
- * procedure during preemption fails with a return code of BLE_HS_EPREEMPTED.
- */
 void
-ble_gap_preempt(void)
+ble_gap_preempt_no_lock(void)
 {
     int rc;
     int i;
 
     (void)rc;
-    (void) i;
-
-    ble_hs_lock();
-
-    BLE_HS_DBG_ASSERT(!ble_gap_is_preempted());
+    (void)i;
 
 #if NIMBLE_BLE_ADVERTISE
 #if MYNEWT_VAL(BLE_EXT_ADV)
@@ -4665,13 +4670,47 @@ ble_gap_preempt(void)
         ble_gap_master.preempted_op = BLE_GAP_OP_M_DISC;
     }
 #endif
+}
+
+/**
+ * Aborts all active GAP procedures and prevents new ones from being started.
+ * This function is used to ensure an idle GAP so that the controller's
+ * resolving list can be modified.  When done accessing the resolving list, the
+ * caller must call `ble_gap_preempt_done()` to permit new GAP procedures.
+ *
+ * On preemption, all aborted GAP procedures are reported with a status or
+ * reason code of BLE_HS_EPREEMPTED.  An attempt to initiate a new GAP
+ * procedure during preemption fails with a return code of BLE_HS_EPREEMPTED.
+ */
+void
+ble_gap_preempt(void)
+{
+    ble_hs_lock();
+
+    BLE_HS_DBG_ASSERT(!ble_gap_is_preempted());
+    ble_gap_preempt_no_lock();
+
+    ble_hs_unlock();
+}
+
+/**
+ * Preempts the GAP if it is not already preempted.
+ */
+void
+ble_gap_ensure_preempted(void)
+{
+    ble_hs_lock();
+
+    if (!ble_gap_is_preempted()) {
+        ble_gap_preempt_no_lock();
+    }
 
     ble_hs_unlock();
 }
 
 /**
  * Takes GAP out of the preempted state, allowing new GAP procedures to be
- * initiaited.  This function should only be called after a call to
+ * initiated.  This function should only be called after a call to
  * `ble_gap_preempt()`.
  */
 
diff --git a/nimble/host/src/ble_gap_priv.h b/nimble/host/src/ble_gap_priv.h
index a789f39..0a566c6 100644
--- a/nimble/host/src/ble_gap_priv.h
+++ b/nimble/host/src/ble_gap_priv.h
@@ -110,6 +110,7 @@ int ble_gap_repeat_pairing_event(const struct ble_gap_repeat_pairing *rp);
 int ble_gap_master_in_progress(void);
 
 void ble_gap_preempt(void);
+void ble_gap_ensure_preempted(void);
 void ble_gap_preempt_done(void);
 
 void ble_gap_conn_broken(uint16_t conn_handle, int reason);
diff --git a/nimble/host/src/ble_hs.c b/nimble/host/src/ble_hs.c
index edfd88f..3ee1757 100644
--- a/nimble/host/src/ble_hs.c
+++ b/nimble/host/src/ble_hs.c
@@ -57,6 +57,7 @@ static struct ble_npl_event ble_hs_ev_start_stage1;
 static struct ble_npl_event ble_hs_ev_start_stage2;
 
 uint8_t ble_hs_sync_state;
+uint8_t ble_hs_enabled_state;
 static int ble_hs_reset_reason;
 
 #define BLE_HS_SYNC_RETRY_TIMEOUT_MS    100 /* ms */
@@ -67,7 +68,7 @@ static void *ble_hs_parent_task;
  * Handles unresponsive timeouts and periodic retries in case of resource
  * shortage.
  */
-static struct ble_npl_callout ble_hs_timer_timer;
+static struct ble_npl_callout ble_hs_timer;
 
 /* Shared queue that the host uses for work items. */
 static struct ble_npl_eventq *ble_hs_evq;
@@ -307,6 +308,12 @@ ble_hs_clear_rx_queue(void)
 }
 
 int
+ble_hs_enabled(void)
+{
+    return ble_hs_enabled_state == BLE_HS_ENABLED_STATE_ON;
+}
+
+int
 ble_hs_synced(void)
 {
     return ble_hs_sync_state == BLE_HS_SYNC_STATE_GOOD;
@@ -399,25 +406,35 @@ ble_hs_timer_exp(struct ble_npl_event *ev)
 {
     int32_t ticks_until_next;
 
-    if (!ble_hs_sync_state) {
-        ble_hs_reset();
-        return;
-    }
+    switch (ble_hs_sync_state) {
+    case BLE_HS_SYNC_STATE_GOOD:
+        ticks_until_next = ble_gattc_timer();
+        ble_hs_timer_sched(ticks_until_next);
+
+        ticks_until_next = ble_gap_timer();
+        ble_hs_timer_sched(ticks_until_next);
+
+        ticks_until_next = ble_l2cap_sig_timer();
+        ble_hs_timer_sched(ticks_until_next);
 
-    ticks_until_next = ble_gattc_timer();
-    ble_hs_timer_sched(ticks_until_next);
+        ticks_until_next = ble_sm_timer();
+        ble_hs_timer_sched(ticks_until_next);
 
-    ticks_until_next = ble_gap_timer();
-    ble_hs_timer_sched(ticks_until_next);
+        ticks_until_next = ble_hs_conn_timer();
+        ble_hs_timer_sched(ticks_until_next);
+        break;
 
-    ticks_until_next = ble_l2cap_sig_timer();
-    ble_hs_timer_sched(ticks_until_next);
+    case BLE_HS_SYNC_STATE_BAD:
+        ble_hs_reset();
+        break;
 
-    ticks_until_next = ble_sm_timer();
-    ble_hs_timer_sched(ticks_until_next);
+    case BLE_HS_SYNC_STATE_BRINGUP:
+    default:
+        /* The timer should not be set in this state. */
+        assert(0);
+        break;
+    }
 
-    ticks_until_next = ble_hs_conn_timer();
-    ble_hs_timer_sched(ticks_until_next);
 }
 
 static void
@@ -425,8 +442,12 @@ ble_hs_timer_reset(uint32_t ticks)
 {
     int rc;
 
-    rc = ble_npl_callout_reset(&ble_hs_timer_timer, ticks);
-    BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+    if (!ble_hs_enabled()) {
+        ble_npl_callout_stop(&ble_hs_timer);
+    } else {
+        rc = ble_npl_callout_reset(&ble_hs_timer, ticks);
+        BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+    }
 }
 
 static void
@@ -442,9 +463,9 @@ ble_hs_timer_sched(int32_t ticks_from_now)
      * sooner than the previous expiration time.
      */
     abs_time = ble_npl_time_get() + ticks_from_now;
-    if (!ble_npl_callout_is_active(&ble_hs_timer_timer) ||
+    if (!ble_npl_callout_is_active(&ble_hs_timer) ||
             ((ble_npl_stime_t)(abs_time -
-                               ble_npl_callout_get_ticks(&ble_hs_timer_timer))) < 0) {
+                               ble_npl_callout_get_ticks(&ble_hs_timer))) < 0) {
         ble_hs_timer_reset(ticks_from_now);
     }
 }
@@ -595,15 +616,40 @@ ble_hs_start(void)
 {
     int rc;
 
+    ble_hs_lock();
+    switch (ble_hs_enabled_state) {
+    case BLE_HS_ENABLED_STATE_ON:
+        rc = BLE_HS_EALREADY;
+        break;
+
+    case BLE_HS_ENABLED_STATE_STOPPING:
+        rc = BLE_HS_EBUSY;
+        break;
+
+    case BLE_HS_ENABLED_STATE_OFF:
+        ble_hs_enabled_state = BLE_HS_ENABLED_STATE_ON;
+        rc = 0;
+        break;
+
+    default:
+        assert(0);
+        rc = BLE_HS_EUNKNOWN;
+        break;
+    }
+    ble_hs_unlock();
+
+    if (rc != 0) {
+        return rc;
+    }
+
     ble_hs_parent_task = ble_npl_get_current_task_id();
 
     /* Stop the timer just in case the host was already running (e.g., unit
      * tests).
      */
-    ble_npl_callout_stop(&ble_hs_timer_timer);
+    ble_npl_callout_stop(&ble_hs_timer);
 
-    ble_npl_callout_init(&ble_hs_timer_timer, ble_hs_evq,
-                    ble_hs_timer_exp, NULL);
+    ble_npl_callout_init(&ble_hs_timer, ble_hs_evq, ble_hs_timer_exp, NULL);
 
     rc = ble_gatts_start();
     if (rc != 0) {
@@ -681,6 +727,7 @@ ble_hs_init(void)
      * bss.
      */
     ble_hs_reset_reason = 0;
+    ble_hs_enabled_state = BLE_HS_ENABLED_STATE_OFF;
 
     ble_npl_event_init(&ble_hs_ev_tx_notifications, ble_hs_event_tx_notify,
                        NULL);
@@ -718,6 +765,8 @@ ble_hs_init(void)
     rc = ble_gatts_init();
     SYSINIT_PANIC_ASSERT(rc == 0);
 
+    ble_hs_stop_init();
+
     ble_mqueue_init(&ble_hs_rx_q, ble_hs_event_rx_data, NULL);
 
     rc = stats_init_and_reg(
diff --git a/nimble/host/src/ble_hs_priv.h b/nimble/host/src/ble_hs_priv.h
index 81e9519..f7d3cd4 100644
--- a/nimble/host/src/ble_hs_priv.h
+++ b/nimble/host/src/ble_hs_priv.h
@@ -60,6 +60,10 @@ struct os_event;
 #define BLE_HS_SYNC_STATE_BRINGUP       1
 #define BLE_HS_SYNC_STATE_GOOD          2
 
+#define BLE_HS_ENABLED_STATE_OFF        0
+#define BLE_HS_ENABLED_STATE_STOPPING   1
+#define BLE_HS_ENABLED_STATE_ON         2
+
 #if NIMBLE_BLE_CONNECT
 #define BLE_HS_MAX_CONNECTIONS MYNEWT_VAL(BLE_MAX_CONNECTIONS)
 #else
@@ -90,6 +94,7 @@ extern STATS_SECT_DECL(ble_hs_stats) ble_hs_stats;
 
 extern struct os_mbuf_pool ble_hs_mbuf_pool;
 extern uint8_t ble_hs_sync_state;
+extern uint8_t ble_hs_enabled_state;
 
 extern const uint8_t ble_hs_misc_null_addr[6];
 
@@ -125,6 +130,7 @@ void ble_hs_hw_error(uint8_t hw_code);
 void ble_hs_timer_resched(void);
 void ble_hs_notifications_sched(void);
 struct ble_npl_eventq *ble_hs_evq_get(void);
+void ble_hs_stop_init(void);
 
 struct ble_mqueue {
     STAILQ_HEAD(, os_mbuf_pkthdr) head;
diff --git a/nimble/host/src/ble_hs_stop.c b/nimble/host/src/ble_hs_stop.c
new file mode 100644
index 0000000..2c46188
--- /dev/null
+++ b/nimble/host/src/ble_hs_stop.c
@@ -0,0 +1,218 @@
+/*
+ * 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 <assert.h>
+#include "sysinit/sysinit.h"
+#include "syscfg/syscfg.h"
+#include "ble_hs_priv.h"
+
+static ble_npl_event_fn ble_hs_stop_term_event_cb;
+static struct ble_npl_event ble_hs_stop_term_ev;
+
+static struct ble_gap_event_listener ble_hs_stop_gap_listener;
+
+/**
+ * List of stop listeners.  These are notified when a stop procedure completes.
+ */
+SLIST_HEAD(ble_hs_stop_listener_slist, ble_hs_stop_listener);
+static struct ble_hs_stop_listener_slist ble_hs_stop_listeners;
+
+/**
+ * Called when a stop procedure has completed.
+ */
+static void
+ble_hs_stop_done(int status)
+{
+    struct ble_hs_stop_listener_slist slist;
+    struct ble_hs_stop_listener *listener;
+
+    ble_hs_lock();
+
+    ble_gap_event_listener_unregister(&ble_hs_stop_gap_listener);
+
+    slist = ble_hs_stop_listeners;
+    SLIST_INIT(&ble_hs_stop_listeners);
+
+    ble_hs_enabled_state = BLE_HS_ENABLED_STATE_OFF;
+
+    ble_hs_unlock();
+
+    SLIST_FOREACH(listener, &slist, link) {
+        listener->fn(status, listener->arg);
+    }
+}
+
+/**
+ * Terminates the first open connection.
+ *
+ * If there are no open connections, signals completion of the close procedure.
+ */
+static void
+ble_hs_stop_terminate_next_conn(void)
+{
+    uint16_t handle;
+    int rc;
+
+    handle = ble_hs_atomic_first_conn_handle();
+    if (handle == BLE_HS_CONN_HANDLE_NONE) {
+        /* No open connections.  Signal completion of the stop procedure. */
+        ble_hs_stop_done(0);
+        return;
+    }
+
+    rc = ble_gap_terminate(handle, BLE_ERR_REM_USER_CONN_TERM);
+    if (rc == 0) {
+        /* Terminate procedure successfully initiated.  Let the GAP event
+         * handler deal with the result.
+         */
+    } else {
+        BLE_HS_LOG(ERROR,
+            "ble_hs_stop: failed to terminate connection; rc=%d\n", rc);
+        ble_hs_stop_done(rc);
+    }
+}
+
+/**
+ * Event handler.  Attempts to terminate the first open connection if there is
+ * one.  All additional connections are terminated elsewhere in the GAP event
+ * handler.
+ *
+ * If there are no connections, signals completion of the stop procedure.
+ */
+static void
+ble_hs_stop_term_event_cb(struct ble_npl_event *ev)
+{
+    ble_hs_stop_terminate_next_conn();
+}
+
+/**
+ * GAP event callback.  Listens for connection termination and then terminates
+ * the next one.
+ *
+ * If there are no connections, signals completion of the stop procedure.
+ */
+static int
+ble_hs_stop_gap_event(struct ble_gap_event *event, void *arg)
+{
+    /* Only process connection termination events. */
+    if (event->type == BLE_GAP_EVENT_DISCONNECT ||
+        event->type == BLE_GAP_EVENT_TERM_FAILURE) {
+
+        ble_hs_stop_terminate_next_conn();
+    }
+
+    return 0;
+}
+
+/**
+ * Registers a listener to listen for completion of the current stop procedure.
+ */
+static void
+ble_hs_stop_register_listener(struct ble_hs_stop_listener *listener,
+                              ble_hs_stop_fn *fn, void *arg)
+{
+    BLE_HS_DBG_ASSERT(fn != NULL);
+
+    listener->fn = fn;
+    listener->arg = arg;
+    SLIST_INSERT_HEAD(&ble_hs_stop_listeners, listener, link);
+}
+
+static int
+ble_hs_stop_begin(struct ble_hs_stop_listener *listener,
+                   ble_hs_stop_fn *fn, void *arg)
+{
+    switch (ble_hs_enabled_state) {
+    case BLE_HS_ENABLED_STATE_ON:
+        /* Host is enabled; proceed with the stop procedure. */
+        ble_hs_enabled_state = BLE_HS_ENABLED_STATE_STOPPING;
+        if (listener != NULL) {
+            ble_hs_stop_register_listener(listener, fn, arg);
+        }
+
+        /* Put the host in the "stopping" state and ensure the host timer is
+         * not running.
+         */
+        ble_hs_timer_resched();
+        return 0;
+
+    case BLE_HS_ENABLED_STATE_STOPPING:
+        /* A stop procedure is already in progress.  Just listen for the
+         * procedure's completion.
+         */
+        if (listener != NULL) {
+            ble_hs_stop_register_listener(listener, fn, arg);
+        }
+        return BLE_HS_EBUSY;
+
+    case BLE_HS_ENABLED_STATE_OFF:
+        /* Host already stopped. */
+        return BLE_HS_EALREADY;
+
+    default:
+        assert(0);
+        return BLE_HS_EUNKNOWN;
+    }
+}
+
+int
+ble_hs_stop(struct ble_hs_stop_listener *listener, 
+            ble_hs_stop_fn *fn, void *arg)
+{
+    int rc;
+
+    ble_hs_lock();
+    rc = ble_hs_stop_begin(listener, fn, arg);
+    ble_hs_unlock();
+
+    switch (rc) {
+    case 0:
+        break;
+
+    case BLE_HS_EBUSY:
+        return 0;
+
+    default:
+        return rc;
+    }
+
+    /* Abort all active GAP procedures. */
+    ble_gap_ensure_preempted();
+    ble_gap_preempt_done();
+
+    rc = ble_gap_event_listener_register(&ble_hs_stop_gap_listener,
+                                         ble_hs_stop_gap_event, NULL);
+    if (rc != 0) {
+        return rc;
+    }
+
+    /* Schedule termination of all open connections in the host task.  This is
+     * done even if there are no open connections so that the result of the
+     * stop procedure is signaled in a consistent manner (asynchronously).
+     */
+    ble_npl_eventq_put(ble_hs_evq_get(), &ble_hs_stop_term_ev);
+
+    return 0;
+}
+
+void
+ble_hs_stop_init(void)
+{
+    ble_npl_event_init(&ble_hs_stop_term_ev, ble_hs_stop_term_event_cb, NULL);
+}
diff --git a/nimble/host/test/src/ble_hs_pvcy_test.c b/nimble/host/test/src/ble_hs_pvcy_test.c
index 4a8bf62..0bf982a 100644
--- a/nimble/host/test/src/ble_hs_pvcy_test.c
+++ b/nimble/host/test/src/ble_hs_pvcy_test.c
@@ -136,6 +136,7 @@ ble_hs_pvcy_test_util_start_host(int num_expected_irks)
         ble_hs_pvcy_test_util_add_irk_set_acks(false, false);
     }
 
+    ble_hs_enabled_state = BLE_HS_ENABLED_STATE_OFF;
     rc = ble_hs_start();
     TEST_ASSERT_FATAL(rc == 0);
 
diff --git a/nimble/host/test/src/ble_hs_stop_test.c b/nimble/host/test/src/ble_hs_stop_test.c
new file mode 100644
index 0000000..2be0151
--- /dev/null
+++ b/nimble/host/test/src/ble_hs_stop_test.c
@@ -0,0 +1,200 @@
+/*
+ * 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 <stddef.h>
+#include <errno.h>
+#include <string.h>
+#include "testutil/testutil.h"
+#include "host/ble_hs.h"
+#include "host/ble_hs_test.h"
+#include "ble_hs_test_util.h"
+
+#define BHST_MAX_EVENTS     32
+
+static struct ble_gap_event bhst_events[BHST_MAX_EVENTS];
+static int bhst_num_events;
+
+static struct ble_hs_stop_listener bhst_listener;
+static struct os_sem bhst_sem;
+
+static int
+bhst_gap_event(struct ble_gap_event *event, void *arg)
+{
+    TEST_ASSERT_FATAL(bhst_num_events < BHST_MAX_EVENTS);
+
+    bhst_events[bhst_num_events++] = *event;
+    return 0;
+}
+
+static void
+bhst_stop_cb(int status, void *arg)
+{
+    int rc;
+
+    rc = os_sem_release(&bhst_sem);
+    TEST_ASSERT_FATAL(rc == 0);
+}
+
+TEST_CASE_TASK(ble_hs_stop_test_new_procs)
+{
+    static const struct ble_gap_disc_params disc_params;
+    static const struct ble_gap_adv_params adv_params;
+
+    static const ble_addr_t peer_addr = {
+        BLE_ADDR_PUBLIC,
+        { 1, 2, 3, 4, 5, 6 }
+    };
+
+    int rc;
+
+    rc = os_sem_init(&bhst_sem, 0);
+    TEST_ASSERT_FATAL(rc == 0);
+
+    /* Stop the host and wait for the stop procedure to complete. */
+    ble_hs_test_util_hci_ack_set(
+        BLE_HCI_OP(BLE_HCI_OGF_LE, BLE_HCI_OCF_LE_SET_ADV_ENABLE), 0);
+
+    rc = ble_hs_stop(&bhst_listener, bhst_stop_cb, NULL);
+    TEST_ASSERT_FATAL(rc == 0);
+    rc = os_sem_pend(&bhst_sem, OS_TIMEOUT_NEVER);
+    TEST_ASSERT_FATAL(rc == 0);
+
+    /*** Ensure all GAP procedures fail. */
+
+    /* Advertise. */
+    rc = ble_hs_test_util_adv_start(BLE_OWN_ADDR_PUBLIC, NULL, &adv_params,
+                                    BLE_HS_FOREVER, bhst_gap_event, NULL,
+                                    0, 0);
+    TEST_ASSERT(rc == BLE_HS_EDISABLED);
+
+    /* Discover. */
+    rc = ble_hs_test_util_disc(BLE_OWN_ADDR_PUBLIC, BLE_HS_FOREVER,
+                               &disc_params, bhst_gap_event, NULL, 0, 0);
+    TEST_ASSERT(rc == BLE_HS_EDISABLED);
+
+    /* Connect. */
+    rc = ble_hs_test_util_connect(BLE_OWN_ADDR_PUBLIC, &peer_addr,
+                                  BLE_HS_FOREVER, NULL,
+                                  bhst_gap_event, NULL, 0);
+    TEST_ASSERT(rc == BLE_HS_EDISABLED);
+
+    /*** Restart stack; ensure GAP procedures succeed. */
+
+    ble_hs_test_util_hci_ack_set_startup();
+    ble_hs_sched_start();
+
+    /* Advertise. */
+    rc = ble_hs_test_util_adv_start(BLE_OWN_ADDR_PUBLIC, NULL, &adv_params,
+                                    BLE_HS_FOREVER, bhst_gap_event, NULL,
+                                    0, 0);
+    TEST_ASSERT(rc == 0);
+
+    rc = ble_hs_test_util_adv_stop(0);
+    TEST_ASSERT(rc == 0);
+
+    /* Discover. */
+    rc = ble_hs_test_util_disc(BLE_OWN_ADDR_PUBLIC, BLE_HS_FOREVER,
+                               &disc_params, bhst_gap_event, NULL, 0, 0);
+    TEST_ASSERT(rc == 0);
+
+    rc = ble_hs_test_util_disc_cancel(0);
+    TEST_ASSERT(rc == 0);
+
+    /* Connect. */
+    rc = ble_hs_test_util_connect(BLE_OWN_ADDR_PUBLIC, &peer_addr,
+                                  BLE_HS_FOREVER, NULL,
+                                  bhst_gap_event, NULL, 0);
+    TEST_ASSERT(rc == 0);
+
+    rc = ble_hs_test_util_conn_cancel(0);
+    TEST_ASSERT(rc == 0);
+}
+
+TEST_CASE_TASK(ble_hs_stop_test_cur_procs)
+{
+    static const struct ble_gap_disc_params disc_params;
+    static const struct ble_gap_adv_params adv_params;
+
+    int rc;
+
+    rc = os_sem_init(&bhst_sem, 0);
+    TEST_ASSERT_FATAL(rc == 0);
+
+    /* Advertise. */
+    rc = ble_hs_test_util_adv_start(BLE_OWN_ADDR_PUBLIC, NULL, &adv_params,
+                                    BLE_HS_FOREVER, bhst_gap_event, NULL,
+                                    0, 0);
+    TEST_ASSERT(rc == 0);
+
+    /* Discover. */
+    rc = ble_hs_test_util_disc(BLE_OWN_ADDR_PUBLIC, BLE_HS_FOREVER,
+                               &disc_params, bhst_gap_event, NULL, 0, 0);
+    TEST_ASSERT(rc == 0);
+
+    /* Preload the host with HCI acks for the cancel commands that will be sent
+     * automatically when the host stops.
+     */
+    ble_hs_test_util_hci_ack_set(
+        BLE_HS_TEST_UTIL_LE_OPCODE(BLE_HCI_OCF_LE_SET_ADV_ENABLE),
+        0);
+    ble_hs_test_util_hci_ack_append(
+        ble_hs_hci_util_opcode_join(BLE_HCI_OGF_LE,
+                                    BLE_HCI_OCF_LE_SET_SCAN_ENABLE),
+        0);
+
+    /* Stop the host and wait for the stop procedure to complete. */
+    bhst_num_events = 0;
+    rc = ble_hs_stop(&bhst_listener, bhst_stop_cb, NULL);
+    TEST_ASSERT_FATAL(rc == 0);
+    rc = os_sem_pend(&bhst_sem, OS_TIMEOUT_NEVER);
+    TEST_ASSERT_FATAL(rc == 0);
+
+    /* Ensure the GAP procedure cancellations were reported. */
+    TEST_ASSERT_FATAL(bhst_num_events == 2);
+    TEST_ASSERT(bhst_events[0].type == BLE_GAP_EVENT_ADV_COMPLETE);
+    TEST_ASSERT(bhst_events[0].adv_complete.reason == BLE_HS_EPREEMPTED);
+    TEST_ASSERT(bhst_events[1].type == BLE_GAP_EVENT_DISC_COMPLETE);
+    TEST_ASSERT(bhst_events[1].disc_complete.reason == BLE_HS_EPREEMPTED);
+}
+
+static void
+bhst_pre_test(void *arg)
+{
+    ble_hs_test_util_init_no_sysinit_no_start();
+
+    /* Preload the host with HCI acks for the startup sequence. */
+    ble_hs_test_util_hci_ack_set_startup();
+}
+
+TEST_SUITE(ble_hs_stop_test_suite)
+{
+    tu_suite_set_post_test_cb(ble_hs_test_util_post_test, NULL);
+    tu_suite_set_pre_test_cb(bhst_pre_test, NULL);
+
+    ble_hs_stop_test_new_procs();
+    ble_hs_stop_test_cur_procs();
+}
+
+int
+ble_stop_test_all(void)
+{
+    ble_hs_stop_test_suite();
+
+    return tu_any_failed;
+}
diff --git a/nimble/host/test/src/ble_hs_test.c b/nimble/host/test/src/ble_hs_test.c
index 021b268..db7bfb8 100644
--- a/nimble/host/test/src/ble_hs_test.c
+++ b/nimble/host/test/src/ble_hs_test.c
@@ -53,6 +53,7 @@ main(int argc, char **argv)
     ble_l2cap_test_all();
     ble_os_test_all();
     ble_sm_test_all();
+    ble_stop_test_all();
     ble_store_test_all();
     ble_uuid_test_all();
 
diff --git a/nimble/host/test/src/ble_hs_test_util.c b/nimble/host/test/src/ble_hs_test_util.c
index afd7a7d..b2fef75 100644
--- a/nimble/host/test/src/ble_hs_test_util.c
+++ b/nimble/host/test/src/ble_hs_test_util.c
@@ -1991,10 +1991,8 @@ ble_hs_test_util_reg_svcs(const struct ble_gatt_svc_def *svcs,
 
 
 void
-ble_hs_test_util_init_no_start(void)
+ble_hs_test_util_init_no_sysinit_no_start(void)
 {
-    sysinit();
-
     STAILQ_INIT(&ble_hs_test_util_prev_tx_queue);
     ble_hs_test_util_prev_tx_cur = NULL;
 
@@ -2005,6 +2003,8 @@ ble_hs_test_util_init_no_start(void)
 
     ble_hs_test_util_hci_ack_set_startup();
 
+    ble_hs_enabled_state = BLE_HS_ENABLED_STATE_OFF;
+
     ble_hs_max_services = 16;
     ble_hs_max_client_configs = 32;
     ble_hs_max_attrs = 64;
@@ -2019,6 +2019,13 @@ ble_hs_test_util_init_no_start(void)
 }
 
 void
+ble_hs_test_util_init_no_start(void)
+{
+    sysinit();
+    ble_hs_test_util_init_no_sysinit_no_start();
+}
+
+void
 ble_hs_test_util_init(void)
 {
     int rc;
diff --git a/nimble/host/test/src/ble_hs_test_util.h b/nimble/host/test/src/ble_hs_test_util.h
index 2640adf..cb5d148 100644
--- a/nimble/host/test/src/ble_hs_test_util.h
+++ b/nimble/host/test/src/ble_hs_test_util.h
@@ -275,6 +275,7 @@ void ble_hs_test_util_reg_svcs(const struct ble_gatt_svc_def *svcs,
                                ble_gatt_register_fn *reg_cb,
                                void *cb_arg);
 void ble_hs_test_util_init_no_start(void);
+void ble_hs_test_util_init_no_sysinit_no_start(void);
 void ble_hs_test_util_init(void);
 
 #ifdef __cplusplus
diff --git a/porting/nimble/Makefile.defs b/porting/nimble/Makefile.defs
index f5396aa..d3c3cd6 100644
--- a/porting/nimble/Makefile.defs
+++ b/porting/nimble/Makefile.defs
@@ -64,6 +64,7 @@ NIMBLE_SRC := \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_hs_misc.c \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_hs_pvcy.c \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_hs_startup.c \
+	$(NIMBLE_ROOT)/nimble/host/src/ble_hs_stop.c \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_ibeacon.c \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_l2cap.c \
 	$(NIMBLE_ROOT)/nimble/host/src/ble_l2cap_coc.c \