You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by ma...@apache.org on 2016/04/28 02:14:21 UTC

[22/50] [abbrv] incubator-mynewt-core git commit: ble host - separate blocking HCI from rx

ble host - separate blocking HCI from rx


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/151139ce
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/tree/151139ce
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/diff/151139ce

Branch: refs/heads/master
Commit: 151139ced61ffdeb339977844be5bb99ed0986f3
Parents: d200094
Author: Christopher Collins <cc...@apache.org>
Authored: Wed Apr 20 12:42:32 2016 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Wed Apr 20 16:11:38 2016 -0700

----------------------------------------------------------------------
 net/nimble/host/include/host/host_hci.h |   1 -
 net/nimble/host/src/ble_gap.c           |  32 +--
 net/nimble/host/src/ble_hci_cmd.c       | 293 +++++++++++++++++++++++++++
 net/nimble/host/src/ble_hci_util.c      |   6 +-
 net/nimble/host/src/ble_hs.c            |   2 +-
 net/nimble/host/src/ble_hs_priv.h       |  10 +-
 net/nimble/host/src/ble_hs_startup.c    |  10 +-
 net/nimble/host/src/ble_l2cap_sm.c      |   4 +-
 net/nimble/host/src/host_hci.c          | 235 +--------------------
 9 files changed, 327 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/include/host/host_hci.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/include/host/host_hci.h b/net/nimble/host/include/host/host_hci.h
index 1b8d61f..8011f79 100644
--- a/net/nimble/host/include/host/host_hci.h
+++ b/net/nimble/host/include/host/host_hci.h
@@ -129,6 +129,5 @@ int host_hci_data_rx(struct os_mbuf *om);
 int host_hci_data_tx(struct ble_hs_conn *connection, struct os_mbuf *om);
 
 void host_hci_timer_set(void);
-void host_hci_init(void);
 
 #endif /* H_HOST_HCI_ */

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/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 7e87661..5a556d1 100644
--- a/net/nimble/host/src/ble_gap.c
+++ b/net/nimble/host/src/ble_gap.c
@@ -1013,7 +1013,7 @@ ble_gap_wl_tx_add(struct ble_gap_white_entry *entry)
         return rc;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1032,7 +1032,7 @@ ble_gap_wl_tx_clear(void)
     int rc;
 
     host_hci_cmd_build_le_clear_whitelist(buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1114,7 +1114,7 @@ ble_gap_adv_disable_tx(void)
     int rc;
 
     host_hci_cmd_build_le_set_adv_enable(0, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1204,7 +1204,7 @@ ble_gap_adv_enable_tx(void)
 
     host_hci_cmd_build_le_set_adv_enable(1, buf, sizeof buf);
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1229,7 +1229,7 @@ ble_gap_adv_rsp_data_tx(void)
         return rc;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1294,7 +1294,7 @@ ble_gap_adv_data_tx(void)
         return rc;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1335,7 +1335,7 @@ ble_gap_adv_params_tx(struct hci_adv_params *adv_params)
         return rc;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1582,7 +1582,7 @@ ble_gap_disc_tx_disable(void)
     int rc;
 
     host_hci_cmd_build_le_set_scan_enable(0, 0, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1601,7 +1601,7 @@ ble_gap_disc_tx_enable(void)
     int rc;
 
     host_hci_cmd_build_le_set_scan_enable(1, 0, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1628,7 +1628,7 @@ ble_gap_disc_tx_params(uint8_t scan_type, uint8_t filter_policy)
         buf, sizeof buf);
     BLE_HS_DBG_ASSERT_EVAL(rc == 0);
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1770,7 +1770,7 @@ ble_gap_conn_create_tx(int addr_type, uint8_t *addr,
         return BLE_HS_EUNKNOWN;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1884,7 +1884,7 @@ ble_gap_terminate(uint16_t conn_handle)
 
     host_hci_cmd_build_disconnect(conn_handle, BLE_ERR_REM_USER_CONN_TERM,
                                   buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         goto done;
     }
@@ -1927,7 +1927,7 @@ ble_gap_cancel(void)
     BLE_HS_LOG(INFO, "GAP procedure initiated: cancel connection\n");
 
     host_hci_cmd_build_le_create_conn_cancel(buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         goto done;
     }
@@ -1965,7 +1965,7 @@ ble_gap_tx_param_pos_reply(uint16_t conn_handle,
     pos_reply.max_ce_len = params->max_ce_len;
 
     host_hci_cmd_build_le_conn_param_reply(&pos_reply, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -1984,7 +1984,7 @@ ble_gap_tx_param_neg_reply(uint16_t conn_handle, uint8_t reject_reason)
     neg_reply.reason = reject_reason;
 
     host_hci_cmd_build_le_conn_param_neg_reply(&neg_reply, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -2079,7 +2079,7 @@ ble_gap_update_tx(uint16_t conn_handle, struct ble_gap_upd_params *params)
         return rc;
     }
 
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_hci_cmd.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hci_cmd.c b/net/nimble/host/src/ble_hci_cmd.c
new file mode 100644
index 0000000..76b7b0e
--- /dev/null
+++ b/net/nimble/host/src/ble_hci_cmd.c
@@ -0,0 +1,293 @@
+/**
+ * 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 <stdint.h>
+#include <string.h>
+#include <errno.h>
+#include <stdio.h>
+#include "os/os.h"
+#include "ble_hs_priv.h"
+#include "host_dbg.h"
+
+#define BLE_HCI_CMD_TIMEOUT     (OS_TICKS_PER_SEC)
+
+static struct os_mutex ble_hci_cmd_mutex;
+static struct os_sem ble_hci_cmd_sem;
+static uint8_t *ble_hci_cmd_ack_ev;
+
+#if PHONY_HCI_ACKS
+static ble_hci_cmd_phony_ack_fn *ble_hci_cmd_phony_ack_cb;
+#endif
+
+#if PHONY_HCI_ACKS
+void
+ble_hci_set_phony_ack_cb(ble_hci_cmd_phony_ack_fn *cb)
+{
+    ble_hci_cmd_phony_ack_cb = cb;
+}
+#endif
+
+static void
+ble_hci_cmd_lock(void)
+{
+    int rc;
+
+    rc = os_mutex_pend(&ble_hci_cmd_mutex, 0xffffffff);
+    BLE_HS_DBG_ASSERT_EVAL(rc == 0 || rc == OS_NOT_STARTED);
+}
+
+static void
+ble_hci_cmd_unlock(void)
+{
+    int rc;
+
+    rc = os_mutex_release(&ble_hci_cmd_mutex);
+    BLE_HS_DBG_ASSERT_EVAL(rc == 0 || rc == OS_NOT_STARTED);
+}
+
+static int
+ble_hci_cmd_rx_cmd_complete(uint8_t event_code, uint8_t *data, int len,
+                         struct ble_hci_ack *out_ack)
+{
+    uint16_t opcode;
+    uint8_t *params;
+    uint8_t params_len;
+    uint8_t num_pkts;
+
+    if (len < BLE_HCI_EVENT_CMD_COMPLETE_HDR_LEN) {
+        /* XXX: Increment stat. */
+        return BLE_HS_EMSGSIZE;
+    }
+
+    num_pkts = data[2];
+    opcode = le16toh(data + 3);
+    params = data + 5;
+
+    /* XXX: Process num_pkts field. */
+    (void)num_pkts;
+
+    out_ack->bha_opcode = opcode;
+
+    params_len = len - BLE_HCI_EVENT_CMD_COMPLETE_HDR_LEN;
+    if (params_len > 0) {
+        out_ack->bha_status = BLE_HS_HCI_ERR(params[0]);
+    } else if (opcode == BLE_HCI_OPCODE_NOP) {
+        out_ack->bha_status = 0;
+    } else {
+        out_ack->bha_status = BLE_HS_ECONTROLLER;
+    }
+
+    /* Don't include the status byte in the parameters blob. */
+    if (params_len > 1) {
+        out_ack->bha_params = params + 1;
+        out_ack->bha_params_len = params_len - 1;
+    } else {
+        out_ack->bha_params = NULL;
+        out_ack->bha_params_len = 0;
+    }
+
+    return 0;
+}
+
+static int
+ble_hci_cmd_rx_cmd_status(uint8_t event_code, uint8_t *data, int len,
+                       struct ble_hci_ack *out_ack)
+{
+    uint16_t opcode;
+    uint8_t num_pkts;
+    uint8_t status;
+
+    if (len < BLE_HCI_EVENT_CMD_STATUS_LEN) {
+        /* XXX: Increment stat. */
+        return BLE_HS_EMSGSIZE;
+    }
+
+    status = data[2];
+    num_pkts = data[3];
+    opcode = le16toh(data + 4);
+
+    /* XXX: Process num_pkts field. */
+    (void)num_pkts;
+
+    out_ack->bha_opcode = opcode;
+    out_ack->bha_params = NULL;
+    out_ack->bha_params_len = 0;
+    out_ack->bha_status = BLE_HS_HCI_ERR(status);
+
+    return 0;
+}
+
+static int
+ble_hci_cmd_process_ack(uint8_t *params_buf, uint8_t params_buf_len,
+                        struct ble_hci_ack *out_ack)
+{
+    uint8_t event_code;
+    uint8_t param_len;
+    uint8_t event_len;
+    int rc;
+
+    BLE_HS_DBG_ASSERT(ble_hci_cmd_ack_ev != NULL);
+
+    /* Count events received */
+    STATS_INC(ble_hs_stats, hci_event);
+
+    /* Display to console */
+    host_hci_dbg_event_disp(ble_hci_cmd_ack_ev);
+
+    event_code = ble_hci_cmd_ack_ev[0];
+    param_len = ble_hci_cmd_ack_ev[1];
+    event_len = param_len + 2;
+
+    /* Clear ack fields up front to silence spurious gcc warnings. */
+    memset(out_ack, 0, sizeof *out_ack);
+
+    switch (event_code) {
+    case BLE_HCI_EVCODE_COMMAND_COMPLETE:
+        rc = ble_hci_cmd_rx_cmd_complete(event_code, ble_hci_cmd_ack_ev,
+                                         event_len, out_ack);
+        break;
+
+    case BLE_HCI_EVCODE_COMMAND_STATUS:
+        rc = ble_hci_cmd_rx_cmd_status(event_code, ble_hci_cmd_ack_ev,
+                                       event_len, out_ack);
+        break;
+
+    default:
+        BLE_HS_DBG_ASSERT(0);
+        rc = BLE_HS_EUNKNOWN;
+        break;
+    }
+
+    if (rc == 0) {
+        if (params_buf == NULL) {
+            out_ack->bha_params_len = 0;
+        } else {
+            if (out_ack->bha_params_len > params_buf_len) {
+                out_ack->bha_params_len = params_buf_len;
+                rc = BLE_HS_EMSGSIZE;
+            }
+            memcpy(params_buf, out_ack->bha_params, out_ack->bha_params_len);
+        }
+        out_ack->bha_params = params_buf;
+    }
+
+    os_memblock_put(&g_hci_cmd_pool, ble_hci_cmd_ack_ev);
+    ble_hci_cmd_ack_ev = NULL;
+
+    return rc;
+}
+
+static int
+ble_hci_cmd_wait_for_ack(void)
+{
+    int rc;
+
+#if PHONY_HCI_ACKS
+    if (ble_hci_cmd_phony_ack_cb == NULL) {
+        rc = BLE_HS_ETIMEOUT;
+    } else {
+        BLE_HS_DBG_ASSERT(ble_hci_cmd_ack_ev == NULL);
+        ble_hci_cmd_ack_ev = os_memblock_get(&g_hci_cmd_pool);
+        if (ble_hci_cmd_ack_ev == NULL) {
+            rc = BLE_HS_ENOMEM;
+        } else {
+            rc = ble_hci_cmd_phony_ack_cb(ble_hci_cmd_ack_ev, 260);
+        }
+    }
+#else
+    rc = os_sem_pend(&ble_hci_cmd_sem, BLE_HCI_CMD_TIMEOUT);
+#endif
+
+    return rc;
+}
+
+int
+ble_hci_cmd_tx(void *cmd, void *evt_buf, uint8_t evt_buf_len,
+               uint8_t *out_evt_buf_len)
+{
+    struct ble_hci_ack ack;
+    int rc;
+
+    ble_hci_cmd_lock();
+
+    rc = host_hci_cmd_send_buf(cmd);
+    if (rc != 0) {
+        goto done;
+    }
+
+    rc = ble_hci_cmd_wait_for_ack();
+    if (rc != 0) {
+        goto done;
+    }
+
+    rc = ble_hci_cmd_process_ack(evt_buf, evt_buf_len, &ack);
+    if (rc != 0) {
+        goto done;
+    }
+
+    if (out_evt_buf_len != NULL) {
+        *out_evt_buf_len = ack.bha_params_len;
+    }
+
+    rc = ack.bha_status;
+
+done:
+    ble_hci_cmd_unlock();
+    return rc;
+}
+
+int
+ble_hci_cmd_tx_empty_ack(void *cmd)
+{
+    int rc;
+
+    rc = ble_hci_cmd_tx(cmd, NULL, 0, NULL);
+    if (rc != 0) {
+        return rc;
+    }
+
+    return 0;
+}
+
+void
+ble_hci_cmd_rx_ack(uint8_t *ack_ev)
+{
+    int rc;
+
+    if (ble_hci_cmd_ack_ev != NULL) {
+        /* The controller sent two acks.  Free the first one. */
+        rc = os_memblock_put(&g_hci_cmd_pool, ble_hci_cmd_ack_ev);
+        BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+    }
+
+    /* Hand up the acknowledgement and unblock the application. */
+    ble_hci_cmd_ack_ev = ack_ev;
+    os_sem_release(&ble_hci_cmd_sem);
+}
+
+void
+ble_hci_cmd_init(void)
+{
+    int rc;
+
+    rc = os_sem_init(&ble_hci_cmd_sem, 0);
+    BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+
+    rc = os_mutex_init(&ble_hci_cmd_mutex);
+    BLE_HS_DBG_ASSERT_EVAL(rc == 0);
+}

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_hci_util.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hci_util.c b/net/nimble/host/src/ble_hci_util.c
index 9216377..2a661e9 100644
--- a/net/nimble/host/src/ble_hci_util.c
+++ b/net/nimble/host/src/ble_hci_util.c
@@ -30,7 +30,7 @@ ble_hci_util_read_adv_tx_pwr(int8_t *out_tx_pwr)
     int rc;
 
     host_hci_cmd_build_read_adv_pwr(buf, sizeof buf);
-    rc = ble_hci_tx_cmd(buf, out_tx_pwr, 1, &params_len);
+    rc = ble_hci_cmd_tx(buf, out_tx_pwr, 1, &params_len);
     if (rc != 0) {
         return rc;
     }
@@ -59,7 +59,7 @@ ble_hci_util_rand(void *dst, int len)
 
     u8ptr = dst;
     while (len > 0) {
-        rc = ble_hci_tx_cmd(req_buf, rsp_buf, sizeof rsp_buf, &params_len);
+        rc = ble_hci_cmd_tx(req_buf, rsp_buf, sizeof rsp_buf, &params_len);
         if (rc != 0) {
             return rc;
         }
@@ -87,7 +87,7 @@ ble_hci_util_read_rssi(uint16_t conn_handle, int8_t *out_rssi)
     int rc;
 
     host_hci_cmd_build_read_rssi(conn_handle, buf, sizeof buf);
-    rc = ble_hci_tx_cmd(buf, params, 1, &params_len);
+    rc = ble_hci_cmd_tx(buf, params, 1, &params_len);
     if (rc != 0) {
         return rc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_hs.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hs.c b/net/nimble/host/src/ble_hs.c
index 8a65c00..e05a9e9 100644
--- a/net/nimble/host/src/ble_hs.c
+++ b/net/nimble/host/src/ble_hs.c
@@ -329,7 +329,7 @@ ble_hs_init(struct os_eventq *app_evq, struct ble_hs_cfg *cfg)
         goto err;
     }
 
-    host_hci_init();
+    ble_hci_cmd_init();
 
     rc = ble_hs_conn_init();
     if (rc != 0) {

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_hs_priv.h
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hs_priv.h b/net/nimble/host/src/ble_hs_priv.h
index 09e9ef4..9bd3632 100644
--- a/net/nimble/host/src/ble_hs_priv.h
+++ b/net/nimble/host/src/ble_hs_priv.h
@@ -106,14 +106,16 @@ struct os_mbuf *ble_hs_misc_pkthdr(void);
 
 int ble_hs_misc_pullup_base(struct os_mbuf **om, int base_len);
 
-int ble_hci_tx_cmd(void *cmd, void *evt_buf, uint8_t evt_buf_len,
+int ble_hci_cmd_tx(void *cmd, void *evt_buf, uint8_t evt_buf_len,
                    uint8_t *out_evt_buf_len);
-int ble_hci_tx_cmd_empty_ack(void *cmd);
+int ble_hci_cmd_tx_empty_ack(void *cmd);
+void ble_hci_cmd_rx_ack(uint8_t *ack_ev);
+void ble_hci_cmd_init(void);
 
 #if PHONY_HCI_ACKS
-typedef int ble_hci_phony_ack_fn(uint8_t *ack, int ack_buf_len);
+typedef int ble_hci_cmd_phony_ack_fn(uint8_t *ack, int ack_buf_len);
 
-void ble_hci_set_phony_ack_cb(ble_hci_phony_ack_fn *cb);
+void ble_hci_set_phony_ack_cb(ble_hci_cmd_phony_ack_fn *cb);
 #endif
 
 #define BLE_HS_LOG(lvl, ...) \

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_hs_startup.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_hs_startup.c b/net/nimble/host/src/ble_hs_startup.c
index 98be26c..31af1ea 100644
--- a/net/nimble/host/src/ble_hs_startup.c
+++ b/net/nimble/host/src/ble_hs_startup.c
@@ -32,7 +32,7 @@ ble_hs_startup_le_read_sup_f_tx(void)
     int rc;
 
     host_hci_cmd_build_le_read_loc_supp_feat(buf, sizeof buf);
-    rc = ble_hci_tx_cmd(buf, ack_params, sizeof ack_params, &ack_params_len);
+    rc = ble_hci_cmd_tx(buf, ack_params, sizeof ack_params, &ack_params_len);
     if (rc != 0) {
         return rc;
     }
@@ -57,7 +57,7 @@ ble_hs_startup_le_read_buf_sz_tx(void)
     int rc;
 
     host_hci_cmd_build_le_read_buffer_size(buf, sizeof buf);
-    rc = ble_hci_tx_cmd(buf, ack_params, sizeof ack_params, &ack_params_len);
+    rc = ble_hci_cmd_tx(buf, ack_params, sizeof ack_params, &ack_params_len);
     if (rc != 0) {
         return rc;
     }
@@ -85,7 +85,7 @@ ble_hs_startup_le_set_evmask_tx(void)
 
     /* [ Default event set ]. */
     host_hci_cmd_build_le_set_event_mask(0x000000000000001f, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -101,7 +101,7 @@ ble_hs_startup_set_evmask_tx(void)
 
     /* [ Default event set | LE-meta event ]. */
     host_hci_cmd_build_set_event_mask(0x20001fffffffffff, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -116,7 +116,7 @@ ble_hs_startup_reset_tx(void)
     int rc;
 
     host_hci_cmd_build_reset(buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/ble_l2cap_sm.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/ble_l2cap_sm.c b/net/nimble/host/src/ble_l2cap_sm.c
index e385ac6..e052f2e 100644
--- a/net/nimble/host/src/ble_l2cap_sm.c
+++ b/net/nimble/host/src/ble_l2cap_sm.c
@@ -468,7 +468,7 @@ ble_l2cap_sm_start_encrypt_tx(uint16_t conn_handle, uint8_t *ltk)
     memcpy(cmd.long_term_key, ltk, sizeof cmd.long_term_key);
 
     host_hci_cmd_build_le_start_encrypt(&cmd, buf, sizeof buf);
-    rc = ble_hci_tx_cmd_empty_ack(buf);
+    rc = ble_hci_cmd_tx_empty_ack(buf);
     if (rc != 0) {
         return rc;
     }
@@ -489,7 +489,7 @@ ble_l2cap_sm_lt_key_req_reply_tx(uint16_t conn_handle, uint8_t *ltk)
     memcpy(cmd.long_term_key, ltk, 16);
 
     host_hci_cmd_build_le_lt_key_req_reply(&cmd, buf, sizeof buf);
-    rc = ble_hci_tx_cmd(buf, &ack_conn_handle, sizeof ack_conn_handle,
+    rc = ble_hci_cmd_tx(buf, &ack_conn_handle, sizeof ack_conn_handle,
                         &ack_params_len);
     if (rc != 0) {
         return rc;

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/151139ce/net/nimble/host/src/host_hci.c
----------------------------------------------------------------------
diff --git a/net/nimble/host/src/host_hci.c b/net/nimble/host/src/host_hci.c
index 31eb108..4994cb6 100644
--- a/net/nimble/host/src/host_hci.c
+++ b/net/nimble/host/src/host_hci.c
@@ -31,8 +31,6 @@
 _Static_assert(sizeof (struct hci_data_hdr) == BLE_HCI_DATA_HDR_SZ,
                "struct hci_data_hdr must be 4 bytes");
 
-#define BLE_HCI_TIMEOUT       (OS_TICKS_PER_SEC)
-
 typedef int host_hci_event_fn(uint8_t event_code, uint8_t *data, int len);
 static host_hci_event_fn host_hci_rx_disconn_complete;
 static host_hci_event_fn host_hci_rx_encrypt_change;
@@ -181,78 +179,6 @@ host_hci_rx_encrypt_change(uint8_t event_code, uint8_t *data, int len)
 }
 
 static int
-host_hci_rx_cmd_complete(uint8_t event_code, uint8_t *data, int len,
-                         struct ble_hci_ack *out_ack)
-{
-    uint16_t opcode;
-    uint8_t *params;
-    uint8_t params_len;
-    uint8_t num_pkts;
-
-    if (len < BLE_HCI_EVENT_CMD_COMPLETE_HDR_LEN) {
-        /* XXX: Increment stat. */
-        return BLE_HS_EMSGSIZE;
-    }
-
-    num_pkts = data[2];
-    opcode = le16toh(data + 3);
-    params = data + 5;
-
-    /* XXX: Process num_pkts field. */
-    (void)num_pkts;
-
-    out_ack->bha_opcode = opcode;
-
-    params_len = len - BLE_HCI_EVENT_CMD_COMPLETE_HDR_LEN;
-    if (params_len > 0) {
-        out_ack->bha_status = BLE_HS_HCI_ERR(params[0]);
-    } else if (opcode == BLE_HCI_OPCODE_NOP) {
-        out_ack->bha_status = 0;
-    } else {
-        out_ack->bha_status = BLE_HS_ECONTROLLER;
-    }
-
-    /* Don't include the status byte in the parameters blob. */
-    if (params_len > 1) {
-        out_ack->bha_params = params + 1;
-        out_ack->bha_params_len = params_len - 1;
-    } else {
-        out_ack->bha_params = NULL;
-        out_ack->bha_params_len = 0;
-    }
-
-    return 0;
-}
-
-static int
-host_hci_rx_cmd_status(uint8_t event_code, uint8_t *data, int len,
-                       struct ble_hci_ack *out_ack)
-{
-    uint16_t opcode;
-    uint8_t num_pkts;
-    uint8_t status;
-
-    if (len < BLE_HCI_EVENT_CMD_STATUS_LEN) {
-        /* XXX: Increment stat. */
-        return BLE_HS_EMSGSIZE;
-    }
-
-    status = data[2];
-    num_pkts = data[3];
-    opcode = le16toh(data + 4);
-
-    /* XXX: Process num_pkts field. */
-    (void)num_pkts;
-
-    out_ack->bha_opcode = opcode;
-    out_ack->bha_params = NULL;
-    out_ack->bha_params_len = 0;
-    out_ack->bha_status = BLE_HS_HCI_ERR(status);
-
-    return 0;
-}
-
-static int
 host_hci_rx_num_completed_pkts(uint8_t event_code, uint8_t *data, int len)
 {
     uint16_t num_pkts;
@@ -598,21 +524,6 @@ host_hci_os_event_proc(struct os_event *ev)
     return rc;
 }
 
-static uint8_t *ble_hci_ack_ev;
-static struct os_sem ble_hci_sem;
-
-#if PHONY_HCI_ACKS
-static ble_hci_phony_ack_fn *ble_hci_phony_ack_cb;
-#endif
-
-#if PHONY_HCI_ACKS
-void
-ble_hci_set_phony_ack_cb(ble_hci_phony_ack_fn *cb)
-{
-    ble_hci_phony_ack_cb = cb;
-}
-#endif
-
 /* XXX: For now, put this here */
 int
 ble_hci_transport_ctlr_event_send(uint8_t *hci_ev)
@@ -629,16 +540,7 @@ ble_hci_transport_ctlr_event_send(uint8_t *hci_ev)
         if (hci_ev[3] == 0 && hci_ev[4] == 0) {
             enqueue = 1;
         } else {
-            if (ble_hci_ack_ev != NULL) {
-                /* The controller sent two acks.  Free the first one. */
-                BLE_HS_DBG_ASSERT(0);
-
-                err = os_memblock_put(&g_hci_cmd_pool, ble_hci_ack_ev);
-                BLE_HS_DBG_ASSERT_EVAL(err == OS_OK);
-            }
-
-            ble_hci_ack_ev = hci_ev;
-            os_sem_release(&ble_hci_sem);
+            ble_hci_cmd_rx_ack(hci_ev);
             enqueue = 0;
         }
         break;
@@ -668,132 +570,6 @@ ble_hci_transport_ctlr_event_send(uint8_t *hci_ev)
 }
 
 static int
-ble_hci_process_ack(uint8_t *params_buf, uint8_t params_buf_len,
-                    struct ble_hci_ack *out_ack)
-{
-    uint8_t event_code;
-    uint8_t param_len;
-    uint8_t event_len;
-    int rc;
-
-    BLE_HS_DBG_ASSERT(ble_hci_ack_ev != NULL);
-
-    /* Count events received */
-    STATS_INC(ble_hs_stats, hci_event);
-
-    /* Display to console */
-    host_hci_dbg_event_disp(ble_hci_ack_ev);
-
-    event_code = ble_hci_ack_ev[0];
-    param_len = ble_hci_ack_ev[1];
-    event_len = param_len + 2;
-
-    /* Clear ack fields up front to silence spurious gcc warnings. */
-    memset(out_ack, 0, sizeof *out_ack);
-
-    switch (event_code) {
-    case BLE_HCI_EVCODE_COMMAND_COMPLETE:
-        rc = host_hci_rx_cmd_complete(event_code, ble_hci_ack_ev, event_len,
-                                      out_ack);
-        break;
-
-    case BLE_HCI_EVCODE_COMMAND_STATUS:
-        rc = host_hci_rx_cmd_status(event_code, ble_hci_ack_ev, event_len,
-                                    out_ack);
-        break;
-
-    default:
-        BLE_HS_DBG_ASSERT(0);
-        rc = BLE_HS_EUNKNOWN;
-        break;
-    }
-
-    if (rc == 0) {
-        if (params_buf == NULL) {
-            out_ack->bha_params_len = 0;
-        } else {
-            if (out_ack->bha_params_len > params_buf_len) {
-                out_ack->bha_params_len = params_buf_len;
-                rc = BLE_HS_EMSGSIZE;
-            }
-            memcpy(params_buf, out_ack->bha_params, out_ack->bha_params_len);
-        }
-        out_ack->bha_params = params_buf;
-    }
-
-    os_memblock_put(&g_hci_cmd_pool, ble_hci_ack_ev);
-    ble_hci_ack_ev = NULL;
-
-    return rc;
-}
-
-static int
-ble_hci_wait_for_ack(void)
-{
-    int rc;
-
-#if PHONY_HCI_ACKS
-    if (ble_hci_phony_ack_cb == NULL) {
-        rc = BLE_HS_ETIMEOUT;
-    } else {
-        BLE_HS_DBG_ASSERT(ble_hci_ack_ev == NULL);
-        ble_hci_ack_ev = os_memblock_get(&g_hci_cmd_pool);
-        if (ble_hci_ack_ev == NULL) {
-            rc = BLE_HS_ENOMEM;
-        } else {
-            rc = ble_hci_phony_ack_cb(ble_hci_ack_ev, 260);
-        }
-    }
-#else
-    rc = os_sem_pend(&ble_hci_sem, BLE_HCI_TIMEOUT);
-#endif
-
-    return rc;
-}
-
-int
-ble_hci_tx_cmd(void *cmd, void *evt_buf, uint8_t evt_buf_len,
-               uint8_t *out_evt_buf_len)
-{
-    struct ble_hci_ack ack;
-    int rc;
-
-    rc = host_hci_cmd_send_buf(cmd);
-    if (rc != 0) {
-        return rc;
-    }
-
-    rc = ble_hci_wait_for_ack();
-    if (rc != 0) {
-        return rc;
-    }
-
-    rc = ble_hci_process_ack(evt_buf, evt_buf_len, &ack);
-    if (rc != 0) {
-        return rc;
-    }
-
-    if (out_evt_buf_len != NULL) {
-        *out_evt_buf_len = ack.bha_params_len;
-    }
-
-    return ack.bha_status;
-}
-
-int
-ble_hci_tx_cmd_empty_ack(void *cmd)
-{
-    int rc;
-
-    rc = ble_hci_tx_cmd(cmd, NULL, 0, NULL);
-    if (rc != 0) {
-        return rc;
-    }
-
-    return 0;
-}
-
-static int
 host_hci_data_hdr_strip(struct os_mbuf *om, struct hci_data_hdr *hdr)
 {
     int rc;
@@ -928,12 +704,3 @@ host_hci_data_tx(struct ble_hs_conn *connection, struct os_mbuf *om)
 
     return 0;
 }
-
-void
-host_hci_init(void)
-{
-    int rc;
-
-    rc = os_sem_init(&ble_hci_sem, 0);
-    BLE_HS_DBG_ASSERT_EVAL(rc == 0);
-}