You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by we...@apache.org on 2016/04/19 08:30:17 UTC

[1/2] incubator-mynewt-core git commit: MYNEWT-99: Port encryption to nrf51

Repository: incubator-mynewt-core
Updated Branches:
  refs/heads/develop b0a8c8edb -> a590265d3


MYNEWT-99: Port encryption to nrf51

Add encryption support for the nrf51. The nrf51 cannot encrypt
or decrypt frames with payload greater than 27 bytes. This
means if you turn on encryption you will have to change the
maximum LL packet size in the nimble options header file to
27 as the default is 251.


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

Branch: refs/heads/develop
Commit: a590265d37f3bf671969f15cccf8b92828a1da92
Parents: 73ee5e8
Author: William San Filippo <wi...@runtime.io>
Authored: Mon Apr 18 23:25:04 2016 -0700
Committer: William San Filippo <wi...@runtime.io>
Committed: Mon Apr 18 23:30:06 2016 -0700

----------------------------------------------------------------------
 net/nimble/controller/src/ble_ll.c      |   3 +-
 net/nimble/controller/src/ble_ll_ctrl.c |  14 +-
 net/nimble/drivers/nrf51/src/ble_phy.c  | 247 ++++++++++++++++++++++++++-
 net/nimble/drivers/nrf52/src/ble_phy.c  |  40 +++--
 net/nimble/include/nimble/nimble_opt.h  |  52 +++---
 5 files changed, 303 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a590265d/net/nimble/controller/src/ble_ll.c
----------------------------------------------------------------------
diff --git a/net/nimble/controller/src/ble_ll.c b/net/nimble/controller/src/ble_ll.c
index 0ffcfe9..003d594 100644
--- a/net/nimble/controller/src/ble_ll.c
+++ b/net/nimble/controller/src/ble_ll.c
@@ -735,8 +735,7 @@ ble_ll_rx_end(struct os_mbuf *rxpdu, struct ble_mbuf_hdr *ble_hdr)
     chan = ble_hdr->rxinfo.channel;
     crcok = BLE_MBUF_HDR_CRC_OK(ble_hdr);
 
-    ble_ll_log(BLE_LL_LOG_ID_RX_END,
-               rxbuf[0],
+    ble_ll_log(BLE_LL_LOG_ID_RX_END, rxbuf[0],
                ((uint16_t)ble_hdr->rxinfo.flags << 8) | rxbuf[1],
                (BLE_MBUF_HDR_PTR(rxpdu))->end_cputime);
 

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a590265d/net/nimble/controller/src/ble_ll_ctrl.c
----------------------------------------------------------------------
diff --git a/net/nimble/controller/src/ble_ll_ctrl.c b/net/nimble/controller/src/ble_ll_ctrl.c
index 04ddd5b..63b4c98 100644
--- a/net/nimble/controller/src/ble_ll_ctrl.c
+++ b/net/nimble/controller/src/ble_ll_ctrl.c
@@ -31,14 +31,18 @@
 /* To use spec sample data for testing */
 #undef BLE_LL_ENCRYPT_USE_TEST_DATA
 
-/* For console debug to show session key calculation */
+/*
+ * For console debug to show session key calculation. NOTE: if you define
+ * this the stack requirements for the LL task go up considerably. The
+ * default stack will not be enough and must be increased.
+ */
 #undef BLE_LL_ENCRYPT_DEBUG
 #ifdef BLE_LL_ENCRYPT_DEBUG
 #include "console/console.h"
 #endif
 
 /*
- * XXX: TODO
+ * XXX:
  *  1) Do I need to keep track of which procedures have already been done?
  *     Do I need to worry about repeating procedures?
  *  2) Should we create pool of control pdu's?. Dont need more
@@ -52,14 +56,8 @@
  *  5) We are supposed to remember when we do the data length update proc if
  *  the device sent us an unknown rsp. We should not send it another len req.
  *  Implement this how? Through remote supported features?
- *  6) Remember: some procedures dont have timeout rules.
- *  7) Says that we should reset procedure timer whenever a LL control pdu
- *  is queued for transmission. I dont get it... do some procedures send
- *  multiple packets? I guess so.
  *  8) How to count control pdus sent. DO we count enqueued + sent, or only
  *  sent (actually attempted to tx). Do we count failures? How?
- *  9) NOTE: we are not supposed to send a REJECT_IND_EXT unless we know the
- *  slave supports that feature
  */
 
 /*

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a590265d/net/nimble/drivers/nrf51/src/ble_phy.c
----------------------------------------------------------------------
diff --git a/net/nimble/drivers/nrf51/src/ble_phy.c b/net/nimble/drivers/nrf51/src/ble_phy.c
index 4de8e83..bc4a258 100644
--- a/net/nimble/drivers/nrf51/src/ble_phy.c
+++ b/net/nimble/drivers/nrf51/src/ble_phy.c
@@ -18,16 +18,36 @@
  */
 
 #include <stdint.h>
+#include <string.h>
 #include <assert.h>
 #include "os/os.h"
 #include "bsp/cmsis_nvic.h"
 #include "nimble/ble.h"
+#include "nimble/nimble_opt.h"
 #include "controller/ble_phy.h"
 #include "controller/ble_ll.h"
 #include "mcu/nrf51_bitfields.h"
 
+/*
+ * XXX: need to make the copy from mbuf into the PHY data structures 32-bit
+ * copies or we are screwed.
+ */
+
 /* XXX: 4) Make sure RF is higher priority interrupt than schedule */
 
+/*
+ * XXX: Maximum possible transmit time is 1 msec for a 60ppm crystal
+ * and 16ms for a 30ppm crystal! We need to limit PDU size based on
+ * crystal accuracy
+ */
+
+/* The NRF51 does not support encryption for payload size < 27 bytes */
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+#if (NIMBLE_OPT_LL_MAX_PKT_SIZE > 27)
+#error "nrf51 does not support encryption with packet size > 27 bytes!"
+#endif
+#endif
+
 /* To disable all radio interrupts */
 #define NRF_RADIO_IRQ_MASK_ALL  (0x34FF)
 
@@ -55,6 +75,7 @@ struct ble_phy_obj
     uint8_t phy_state;
     uint8_t phy_transition;
     uint8_t phy_rx_started;
+    uint8_t phy_encrypted;
     uint32_t phy_access_address;
     struct os_mbuf *rxpdu;
     void *txend_arg;
@@ -62,9 +83,14 @@ struct ble_phy_obj
 };
 struct ble_phy_obj g_ble_phy_data;
 
+/* XXX: if 27 byte packets desired we can make this smaller */
 /* Global transmit/receive buffer */
 static uint32_t g_ble_phy_txrx_buf[(BLE_PHY_MAX_PDU_LEN + 3) / 4];
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+static uint32_t g_ble_phy_enc_buf[(BLE_PHY_MAX_PDU_LEN + 3) / 4];
+#endif
+
 /* Statistics */
 STATS_SECT_START(ble_phy_stats)
     STATS_SECT_ENTRY(phy_isrs)
@@ -128,6 +154,24 @@ STATS_NAME_END(ble_phy_stats)
  *  bit in the NVIC just to be sure when we disable the PHY.
  */
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+
+/* Per nordic, the number of bytes needed for scratch is 16 + MAX_PKT_SIZE */
+#define NRF_ENC_SCRATCH_WORDS   (((NIMBLE_OPT_LL_MAX_PKT_SIZE + 16) + 3) / 4)
+
+uint32_t g_nrf_encrypt_scratchpad[NRF_ENC_SCRATCH_WORDS];
+
+struct nrf_ccm_data
+{
+    uint8_t key[16];
+    uint64_t pkt_counter;
+    uint8_t dir_bit;
+    uint8_t iv[8];
+} __attribute__((packed));
+
+struct nrf_ccm_data g_nrf_ccm_data;
+#endif
+
 /**
  * ble phy rxpdu get
  *
@@ -185,6 +229,9 @@ ble_phy_isr(void)
     uint32_t irq_en;
     uint32_t state;
     uint32_t wfr_time;
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+    uint8_t *dptr;
+#endif
     struct os_mbuf *rxpdu;
     struct ble_mbuf_hdr *ble_hdr;
 
@@ -197,7 +244,7 @@ ble_phy_isr(void)
         assert(g_ble_phy_data.phy_state == BLE_PHY_STATE_TX);
 
         ble_ll_log(BLE_LL_LOG_ID_PHY_TXEND, (g_ble_phy_txrx_buf[0] >> 8) & 0xFF,
-                   0, NRF_TIMER0->CC[2]);
+                   g_ble_phy_data.phy_encrypted, NRF_TIMER0->CC[2]);
 
         /* Clear events and clear interrupt on disabled event */
         NRF_RADIO->EVENTS_DISABLED = 0;
@@ -205,6 +252,19 @@ ble_phy_isr(void)
         NRF_RADIO->EVENTS_END = 0;
         state = NRF_RADIO->SHORTS;
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+        /*
+         * XXX: not sure what to do. We had a HW error during transmission.
+         * For now I just count a stat but continue on like all is good.
+         */
+        if (g_ble_phy_data.phy_encrypted) {
+            if (NRF_CCM->EVENTS_ERROR) {
+                STATS_INC(ble_phy_stats, tx_hw_err);
+                NRF_CCM->EVENTS_ERROR = 0;
+            }
+        }
+#endif
+
         transition = g_ble_phy_data.phy_transition;
         if (transition == BLE_PHY_TRANSITION_TX_RX) {
             /* Clear the rx started flag */
@@ -212,7 +272,23 @@ ble_phy_isr(void)
 
             /* Packet pointer needs to be reset. */
             if (g_ble_phy_data.rxpdu != NULL) {
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+                if (g_ble_phy_data.phy_encrypted) {
+                    NRF_RADIO->PACKETPTR = (uint32_t)&g_ble_phy_enc_buf[0];
+                    NRF_CCM->INPTR = (uint32_t)&g_ble_phy_enc_buf[0];
+                    NRF_CCM->OUTPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+                    NRF_CCM->SCRATCHPTR = (uint32_t)&g_nrf_encrypt_scratchpad[0];
+                    NRF_CCM->MODE = CCM_MODE_MODE_Decryption;
+                    NRF_CCM->CNFPTR = (uint32_t)&g_nrf_ccm_data;
+                    NRF_CCM->SHORTS = 0;
+                    NRF_CCM->EVENTS_ENDCRYPT = 0;
+                    NRF_PPI->CHENSET = PPI_CHEN_CH24_Msk | PPI_CHEN_CH25_Msk;
+                } else {
+                    NRF_RADIO->PACKETPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+                }
+#else
                 NRF_RADIO->PACKETPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+#endif
 
                 /* I want to know when 1st byte received (after address) */
                 NRF_RADIO->BCC = 8; /* in bits */
@@ -322,7 +398,9 @@ ble_phy_isr(void)
         assert(NRF_RADIO->EVENTS_RSSIEND != 0);
         ble_hdr->rxinfo.rssi = -1 * NRF_RADIO->RSSISAMPLE;
         ble_hdr->end_cputime = NRF_TIMER0->CC[2];
-
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+        dptr = g_ble_phy_data.rxpdu->om_data;
+#endif
         /* Count PHY crc errors and valid packets */
         crcok = (uint8_t)NRF_RADIO->CRCSTATUS;
         if (!crcok) {
@@ -330,11 +408,53 @@ ble_phy_isr(void)
         } else {
             STATS_INC(ble_phy_stats, rx_valid);
             ble_hdr->rxinfo.flags |= BLE_MBUF_HDR_F_CRC_OK;
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+            if (g_ble_phy_data.phy_encrypted) {
+                /* Only set MIC failure flag if frame is not zero length */
+                if ((dptr[1] != 0) && (NRF_CCM->MICSTATUS == 0)) {
+                    ble_hdr->rxinfo.flags |= BLE_MBUF_HDR_F_MIC_FAILURE;
+                }
+
+                /*
+                 * XXX: not sure how to deal with this. This should not
+                 * be a MIC failure but we should not hand it up. I guess
+                 * this is just some form of rx error and that is how we
+                 * handle it? For now, just set CRC error flags
+                 */
+                if (NRF_CCM->EVENTS_ERROR) {
+                    STATS_INC(ble_phy_stats, rx_hw_err);
+                    ble_hdr->rxinfo.flags &= ~BLE_MBUF_HDR_F_CRC_OK;
+                }
+
+                /*
+                 * XXX: This is a total hack work-around for now but I dont
+                 * know what else to do. If ENDCRYPT is not set and we are
+                 * encrypted we need to not trust this frame and drop it.
+                 */
+                if (NRF_CCM->EVENTS_ENDCRYPT == 0) {
+                    STATS_INC(ble_phy_stats, rx_hw_err);
+                    ble_hdr->rxinfo.flags &= ~BLE_MBUF_HDR_F_CRC_OK;
+                }
+            }
+#endif
         }
 
         /* Call Link Layer receive payload function */
         rxpdu = g_ble_phy_data.rxpdu;
         g_ble_phy_data.rxpdu = NULL;
+
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+        if (g_ble_phy_data.phy_encrypted) {
+            /*
+             * XXX: This is a horrible ugly hack to deal with the RAM S1 byte.
+             * This should get fixed as we should not be handing up the header
+             * and length as part of the pdu.
+             */
+            dptr[2] = dptr[1];
+            dptr[1] = dptr[0];
+            rxpdu->om_data += 1;
+        }
+#endif
         rc = ble_ll_rx_end(rxpdu, ble_hdr);
         if (rc < 0) {
             /* Disable the PHY. */
@@ -390,6 +510,7 @@ ble_phy_init(void)
     NRF_RADIO->MODE = RADIO_MODE_MODE_Ble_1Mbit;
     NRF_RADIO->PCNF0 = (NRF_LFLEN_BITS << RADIO_PCNF0_LFLEN_Pos) |
                        (NRF_S0_LEN << RADIO_PCNF0_S0LEN_Pos);
+    /* XXX: should maxlen be 251 for encryption? */
     NRF_RADIO->PCNF1 = NRF_MAXLEN |
                        (RADIO_PCNF1_ENDIAN_Little <<  RADIO_PCNF1_ENDIAN_Pos) |
                        (NRF_BALEN << RADIO_PCNF1_BALEN_Pos) |
@@ -414,6 +535,14 @@ ble_phy_init(void)
      */
     NRF_PPI->CHENSET = PPI_CHEN_CH27_Msk;
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+    NRF_CCM->INTENCLR = 0xffffffff;
+    NRF_CCM->SHORTS = CCM_SHORTS_ENDKSGEN_CRYPT_Msk;
+    NRF_CCM->ENABLE = CCM_ENABLE_ENABLE_Enabled;
+    NRF_CCM->EVENTS_ERROR = 0;
+    memset(g_nrf_encrypt_scratchpad, 0, sizeof(g_nrf_encrypt_scratchpad));
+#endif
+
     /* Set isr in vector table and enable interrupt */
     NVIC_SetPriority(RADIO_IRQn, 0);
     NVIC_SetVector(RADIO_IRQn, (uint32_t)ble_phy_isr);
@@ -451,7 +580,15 @@ ble_phy_rx(void)
     }
 
     /* Set packet pointer */
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+    if (g_ble_phy_data.phy_encrypted) {
+        NRF_RADIO->PACKETPTR = (uint32_t)&g_ble_phy_enc_buf[0];
+    } else {
+        NRF_RADIO->PACKETPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+    }
+#else
     NRF_RADIO->PACKETPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+#endif
 
     /* Make sure all interrupts are disabled */
     NRF_RADIO->INTENCLR = NRF_RADIO_IRQ_MASK_ALL;
@@ -464,6 +601,24 @@ ble_phy_rx(void)
     NRF_RADIO->EVENTS_RSSIEND = 0;
     NRF_RADIO->EVENTS_DEVMATCH = 0;
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+    if (g_ble_phy_data.phy_encrypted) {
+        NRF_CCM->INPTR = (uint32_t)&g_ble_phy_enc_buf[0];
+        NRF_CCM->OUTPTR = (uint32_t)g_ble_phy_data.rxpdu->om_data;
+        NRF_CCM->SCRATCHPTR = (uint32_t)&g_nrf_encrypt_scratchpad[0];
+        NRF_CCM->EVENTS_ERROR = 0;
+        NRF_CCM->EVENTS_ENDCRYPT = 0;
+        NRF_CCM->MODE = CCM_MODE_MODE_Decryption;
+        /* XXX: can I just set this once? (i.e per connection)? In other
+           words, only do this during encrypt enable? */
+        NRF_CCM->CNFPTR = (uint32_t)&g_nrf_ccm_data;
+        NRF_CCM->SHORTS = 0;
+        NRF_PPI->CHENSET = PPI_CHEN_CH24_Msk | PPI_CHEN_CH25_Msk;
+    }
+#endif
+
+    /* XXX: could it be that I am late turning on the device? That I need
+       to automatically go from rx to tx always? I dont here */
     /* I want to know when 1st byte received (after address) */
     NRF_RADIO->BCC = 8; /* in bits */
     NRF_RADIO->SHORTS = RADIO_SHORTS_END_DISABLE_Msk |
@@ -485,6 +640,55 @@ ble_phy_rx(void)
     return 0;
 }
 
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+/**
+ * Called to enable encryption at the PHY. Note that this state will persist
+ * in the PHY; in other words, if you call this function you have to call
+ * disable so that future PHY transmits/receives will not be encrypted.
+ *
+ * @param pkt_counter
+ * @param iv
+ * @param key
+ * @param is_master
+ */
+void
+ble_phy_encrypt_enable(uint64_t pkt_counter, uint8_t *iv, uint8_t *key,
+                       uint8_t is_master)
+{
+    memcpy(g_nrf_ccm_data.key, key, 16);
+    g_nrf_ccm_data.pkt_counter = pkt_counter;
+    memcpy(g_nrf_ccm_data.iv, iv, 8);
+    g_nrf_ccm_data.dir_bit = is_master;
+    g_ble_phy_data.phy_encrypted = 1;
+
+    /* Encryption uses LFLEN=5, S1LEN = 3. */
+    NRF_RADIO->PCNF0 = (5 << RADIO_PCNF0_LFLEN_Pos) |
+                       (3 << RADIO_PCNF0_S1LEN_Pos) |
+                       (NRF_S0_LEN << RADIO_PCNF0_S0LEN_Pos);
+}
+
+void
+ble_phy_encrypt_set_pkt_cntr(uint64_t pkt_counter, int dir)
+{
+    g_nrf_ccm_data.pkt_counter = pkt_counter;
+    g_nrf_ccm_data.dir_bit = dir;
+}
+
+void
+ble_phy_encrypt_disable(void)
+{
+    NRF_PPI->CHENCLR = (PPI_CHEN_CH24_Msk | PPI_CHEN_CH25_Msk);
+    NRF_CCM->TASKS_STOP = 1;
+    NRF_CCM->EVENTS_ERROR = 0;
+
+    /* Switch back to normal length */
+    NRF_RADIO->PCNF0 = (NRF_LFLEN_BITS << RADIO_PCNF0_LFLEN_Pos) |
+                       (NRF_S0_LEN << RADIO_PCNF0_S0LEN_Pos);
+
+    g_ble_phy_data.phy_encrypted = 0;
+}
+#endif
+
 void
 ble_phy_set_txend_cb(ble_phy_tx_end_func txend_cb, void *arg)
 {
@@ -525,15 +729,46 @@ ble_phy_tx(struct os_mbuf *txpdu, uint8_t beg_trans, uint8_t end_trans)
         return BLE_PHY_ERR_RADIO_STATE;
     }
 
-    /* Write LL header first */
+#ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
+    if (g_ble_phy_data.phy_encrypted) {
+        /* RAM representation has S0, LENGTH and S1 fields. (3 bytes) */
+        ble_hdr = BLE_MBUF_HDR_PTR(txpdu);
+        dptr = (uint8_t *)&g_ble_phy_enc_buf[0];
+        dptr[0] = ble_hdr->txinfo.hdr_byte;
+        dptr[1] = ble_hdr->txinfo.pyld_len;
+        dptr[2] = 0;
+        dptr += 3;
+
+        NRF_RADIO->PACKETPTR = (uint32_t)&g_ble_phy_txrx_buf[0];
+        NRF_CCM->SHORTS = 1;
+        NRF_CCM->INPTR = (uint32_t)&g_ble_phy_enc_buf[0];
+        NRF_CCM->OUTPTR = (uint32_t)&g_ble_phy_txrx_buf[0];
+        NRF_CCM->SCRATCHPTR = (uint32_t)&g_nrf_encrypt_scratchpad[0];
+        NRF_CCM->EVENTS_ERROR = 0;
+        NRF_CCM->MODE = CCM_MODE_MODE_Encryption;
+        NRF_CCM->CNFPTR = (uint32_t)&g_nrf_ccm_data;
+        NRF_PPI->CHENCLR = PPI_CHEN_CH25_Msk;
+        NRF_PPI->CHENSET = PPI_CHEN_CH24_Msk;
+    } else {
+        /* RAM representation has S0 and LENGTH fields (2 bytes) */
+        ble_hdr = BLE_MBUF_HDR_PTR(txpdu);
+        dptr = (uint8_t *)&g_ble_phy_txrx_buf[0];
+        dptr[0] = ble_hdr->txinfo.hdr_byte;
+        dptr[1] = ble_hdr->txinfo.pyld_len;
+        dptr += 2;
+
+        NRF_RADIO->PACKETPTR = (uint32_t)&g_ble_phy_txrx_buf[0];
+    }
+#else
+    /* RAM representation has S0 and LENGTH fields (2 bytes) */
     ble_hdr = BLE_MBUF_HDR_PTR(txpdu);
     dptr = (uint8_t *)&g_ble_phy_txrx_buf[0];
     dptr[0] = ble_hdr->txinfo.hdr_byte;
     dptr[1] = ble_hdr->txinfo.pyld_len;
     dptr += 2;
 
-    /* Set radio transmit data pointer */
     NRF_RADIO->PACKETPTR = (uint32_t)&g_ble_phy_txrx_buf[0];
+#endif
 
     /* Clear the ready, end and disabled events */
     NRF_RADIO->EVENTS_READY = 0;
@@ -638,8 +873,8 @@ ble_phy_txpwr_get(void)
  * ble phy setchan
  *
  * Sets the logical frequency of the transceiver. The input parameter is the
- * BLE channel index (0 to 39, inclusive). The NRF51 frequency register works
- * like this: logical frequency = 2400 + FREQ (MHz).
+ * BLE channel index (0 to 39, inclusive). The NRF frequency register works like
+ * this: logical frequency = 2400 + FREQ (MHz).
  *
  * Thus, to get a logical frequency of 2402 MHz, you would program the
  * FREQUENCY register to 2.

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a590265d/net/nimble/drivers/nrf52/src/ble_phy.c
----------------------------------------------------------------------
diff --git a/net/nimble/drivers/nrf52/src/ble_phy.c b/net/nimble/drivers/nrf52/src/ble_phy.c
index 2c55d48..fee6043 100644
--- a/net/nimble/drivers/nrf52/src/ble_phy.c
+++ b/net/nimble/drivers/nrf52/src/ble_phy.c
@@ -23,6 +23,7 @@
 #include "os/os.h"
 #include "bsp/cmsis_nvic.h"
 #include "nimble/ble.h"
+#include "nimble/nimble_opt.h"
 #include "controller/ble_phy.h"
 #include "controller/ble_ll.h"
 #include "mcu/nrf52_bitfields.h"
@@ -38,7 +39,7 @@
 #define NRF_RADIO_IRQ_MASK_ALL  (0x34FF)
 
 /*
- * We configure the nrf52 with a 1 byte S0 field, 8 bit length field, and
+ * We configure the nrf with a 1 byte S0 field, 8 bit length field, and
  * zero bit S1 field. The preamble is 8 bits long.
  */
 #define NRF_LFLEN_BITS          (8)
@@ -69,8 +70,10 @@ struct ble_phy_obj
 };
 struct ble_phy_obj g_ble_phy_data;
 
+/* XXX: if 27 byte packets desired we can make this smaller */
 /* Global transmit/receive buffer */
 static uint32_t g_ble_phy_txrx_buf[(BLE_PHY_MAX_PDU_LEN + 3) / 4];
+
 #ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
 static uint32_t g_ble_phy_enc_buf[(BLE_PHY_MAX_PDU_LEN + 3) / 4];
 #endif
@@ -113,8 +116,8 @@ STATS_NAME_END(ble_phy_stats)
  * NOTE:
  * Tested the following to see what would happen:
  *  -> NVIC has radio irq enabled (interrupt # 1, mask 0x2).
- *  -> Set up nrf52 to receive. Clear ADDRESS event register.
- *  -> Enable ADDRESS interrupt on nrf52 by writing to INTENSET.
+ *  -> Set up nrf to receive. Clear ADDRESS event register.
+ *  -> Enable ADDRESS interrupt on nrf5 by writing to INTENSET.
  *  -> Enable RX.
  *  -> Disable interrupts globally using OS_ENTER_CRITICAL().
  *  -> Wait until a packet is received and the ADDRESS event occurs.
@@ -139,9 +142,19 @@ STATS_NAME_END(ble_phy_stats)
  */
 
 #ifdef BLE_LL_CFG_FEAT_LE_ENCRYPTION
-/* NRF requires 43 bytes of scratch for encryption */
-/* XXX: align this? */
-uint32_t g_nrf_encrypt_scratchpad[100];
+
+/*
+ * Per nordic, the number of bytes needed for scratch is 16 + MAX_PKT_SIZE.
+ * However, when I used a smaller size it still overwrote the scratchpad. Until
+ * I figure this out I am just going to allocate 67 words so we have enough
+ * space for 267 bytes of scratch. I used 268 bytes since not sure if this
+ * needs to be aligned and burning a byte is no big deal.
+ */
+//#define NRF_ENC_SCRATCH_WORDS (((NIMBLE_OPT_LL_MAX_PKT_SIZE + 16) + 3) / 4)
+#define NRF_ENC_SCRATCH_WORDS   (67)
+
+uint32_t g_nrf_encrypt_scratchpad[NRF_ENC_SCRATCH_WORDS];
+
 struct nrf_ccm_data
 {
     uint8_t key[16];
@@ -184,7 +197,7 @@ ble_phy_rxpdu_get(void)
 }
 
 static void
-nrf52_wait_disabled(void)
+nrf_wait_disabled(void)
 {
     uint32_t state;
 
@@ -544,7 +557,7 @@ int
 ble_phy_rx(void)
 {
     /* Check radio state */
-    nrf52_wait_disabled();
+    nrf_wait_disabled();
     if (NRF_RADIO->STATE != RADIO_STATE_STATE_Disabled) {
         ble_phy_disable();
         STATS_INC(ble_phy_stats, radio_state_errs);
@@ -594,6 +607,8 @@ ble_phy_rx(void)
     }
 #endif
 
+    /* XXX: could it be that I am late turning on the device? That I need
+       to automatically go from rx to tx always? I dont here */
     /* I want to know when 1st byte received (after address) */
     NRF_RADIO->BCC = 8; /* in bits */
     NRF_RADIO->SHORTS = RADIO_SHORTS_END_DISABLE_Msk |
@@ -637,7 +652,6 @@ ble_phy_encrypt_enable(uint64_t pkt_counter, uint8_t *iv, uint8_t *key,
     g_ble_phy_data.phy_encrypted = 1;
 }
 
-
 void
 ble_phy_encrypt_set_pkt_cntr(uint64_t pkt_counter, int dir)
 {
@@ -676,7 +690,7 @@ ble_phy_tx(struct os_mbuf *txpdu, uint8_t beg_trans, uint8_t end_trans)
     assert(txpdu != NULL);
 
     /* If radio is not disabled, */
-    nrf52_wait_disabled();
+    nrf_wait_disabled();
 
     if (beg_trans == BLE_PHY_TRANSITION_RX_TX) {
         if ((NRF_RADIO->SHORTS & RADIO_SHORTS_DISABLED_TXEN_Msk) == 0) {
@@ -841,8 +855,8 @@ ble_phy_txpwr_get(void)
  * ble phy setchan
  *
  * Sets the logical frequency of the transceiver. The input parameter is the
- * BLE channel index (0 to 39, inclusive). The NRF52 frequency register
- * works like this: logical frequency = 2400 + FREQ (MHz).
+ * BLE channel index (0 to 39, inclusive). The NRF frequency register works like
+ * this: logical frequency = 2400 + FREQ (MHz).
  *
  * Thus, to get a logical frequency of 2402 MHz, you would program the
  * FREQUENCY register to 2.
@@ -864,7 +878,7 @@ ble_phy_setchan(uint8_t chan, uint32_t access_addr, uint32_t crcinit)
         return BLE_PHY_ERR_INV_PARAM;
     }
 
-    /* Get correct nrf52 frequency */
+    /* Get correct frequency */
     if (chan < BLE_PHY_NUM_DATA_CHANS) {
         if (chan < 11) {
             /* Data channel 0 starts at 2404. 0 - 10 are contiguous */

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/a590265d/net/nimble/include/nimble/nimble_opt.h
----------------------------------------------------------------------
diff --git a/net/nimble/include/nimble/nimble_opt.h b/net/nimble/include/nimble/nimble_opt.h
index 3e94943..d008d54 100644
--- a/net/nimble/include/nimble/nimble_opt.h
+++ b/net/nimble/include/nimble/nimble_opt.h
@@ -6,7 +6,7 @@
  * 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,
@@ -44,7 +44,7 @@
 #define NIMBLE_OPT_ROLE_BROADCASTER             1
 #endif
 
-#ifndef NIMBLE_OPT_ROLE_OBSERVER   
+#ifndef NIMBLE_OPT_ROLE_OBSERVER
 #define NIMBLE_OPT_ROLE_OBSERVER                1
 #endif
 
@@ -85,23 +85,23 @@
 #define NIMBLE_OPT_GATT_DISC_ALL_DSCS           1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_READ         
+#ifndef NIMBLE_OPT_GATT_READ
 #define NIMBLE_OPT_GATT_READ                    1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_READ_UUID    
+#ifndef NIMBLE_OPT_GATT_READ_UUID
 #define NIMBLE_OPT_GATT_READ_UUID               1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_READ_LONG    
+#ifndef NIMBLE_OPT_GATT_READ_LONG
 #define NIMBLE_OPT_GATT_READ_LONG               1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_READ_MULT    
+#ifndef NIMBLE_OPT_GATT_READ_MULT
 #define NIMBLE_OPT_GATT_READ_MULT               1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_WRITE_NO_RSP 
+#ifndef NIMBLE_OPT_GATT_WRITE_NO_RSP
 #define NIMBLE_OPT_GATT_WRITE_NO_RSP            1
 #endif
 
@@ -109,23 +109,23 @@
 #define NIMBLE_OPT_GATT_SIGNED_WRITE            1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_WRITE              
+#ifndef NIMBLE_OPT_GATT_WRITE
 #define NIMBLE_OPT_GATT_WRITE                   1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_WRITE_LONG         
+#ifndef NIMBLE_OPT_GATT_WRITE_LONG
 #define NIMBLE_OPT_GATT_WRITE_LONG              1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_WRITE_RELIABLE     
+#ifndef NIMBLE_OPT_GATT_WRITE_RELIABLE
 #define NIMBLE_OPT_GATT_WRITE_RELIABLE          1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_NOTIFY             
+#ifndef NIMBLE_OPT_GATT_NOTIFY
 #define NIMBLE_OPT_GATT_NOTIFY                  1
 #endif
 
-#ifndef NIMBLE_OPT_GATT_INDICATE           
+#ifndef NIMBLE_OPT_GATT_INDICATE
 #define NIMBLE_OPT_GATT_INDICATE                1
 #endif
 
@@ -180,7 +180,7 @@
 #define NIMBLE_OPT_ATT_SVR_EXEC_WRITE           1
 #endif
 
-#ifndef NIMBLE_OPT_ATT_SVR_NOTIFY  
+#ifndef NIMBLE_OPT_ATT_SVR_NOTIFY
 #define NIMBLE_OPT_ATT_SVR_NOTIFY               1
 #endif
 
@@ -190,14 +190,14 @@
 
 /*** CONTROLLER ***/
 
-/* 
- * Sleep clock accuracy (sca). This is the amount of drift in the system during 
+/*
+ * Sleep clock accuracy (sca). This is the amount of drift in the system during
  * when the device is sleeping (in parts per million).
  *
  * NOTE: the master sca is an enumerated value based on the sca. Rather than
  * have a piece of code calculate this value, the developer must set this
  * value based on the value of the SCA using the following table:
- * 
+ *
  *  SCA between 251 and 500 ppm (inclusive); master sca = 0
  *  SCA between 151 and 250 ppm (inclusive); master sca = 1
  *  SCA between 101 and 150 ppm (inclusive); master sca = 2
@@ -206,11 +206,11 @@
  *  SCA between 31 and 50 ppm (inclusive); master sca = 5
  *  SCA between 21 and 30 ppm (inclusive); master sca = 6
  *  SCA between 0 and 20 ppm (inclusive); master sca = 7
- * 
+ *
  *  For example:
  *      if your clock drift is 101 ppm, your master should be set to 2.
  *      if your clock drift is 20, your master sca should be set to 7.
- * 
+ *
  *  The values provided below are merely meant to be an example and should
  *  be replaced by values appropriate for your platform.
  */
@@ -227,7 +227,7 @@
 #define NIMBLE_OPT_LL_TX_PWR_DBM                (0)
 #endif
 
-/* 
+/*
  * Determines the maximum rate at which the controller will send the
  * number of completed packets event to the host. Rate is in os time ticks
  */
@@ -240,7 +240,7 @@
 #define NIMBLE_OPT_LL_MFRG_ID                   (0xFFFF)
 #endif
 
-/* 
+/*
  * Configuration items for the number of duplicate advertisers and the
  * number of advertisers from which we have heard a scan response.
  */
@@ -257,16 +257,20 @@
 #define NIMBLE_OPT_LL_WHITELIST_SIZE            (8)
 #endif
 
-/* 
+/*
  * Data length management definitions for connections. These define the maximum
- * size of the PDU's that will be sent and/or received in a connection. 
+ * size of the PDU's that will be sent and/or received in a connection.
  */
+#ifndef NIMBLE_OPT_LL_MAX_PKT_SIZE
+#define NIMBLE_OPT_LL_MAX_PKT_SIZE              (251)
+#endif
+
 #ifndef NIMBLE_OPT_LL_SUPP_MAX_RX_BYTES
-#define NIMBLE_OPT_LL_SUPP_MAX_RX_BYTES         (251)
+#define NIMBLE_OPT_LL_SUPP_MAX_RX_BYTES         (NIMBLE_OPT_LL_MAX_PKT_SIZE)
 #endif
 
 #ifndef NIMBLE_OPT_LL_SUPP_MAX_TX_BYTES
-#define NIMBLE_OPT_LL_SUPP_MAX_TX_BYTES         (251)
+#define NIMBLE_OPT_LL_SUPP_MAX_TX_BYTES         (NIMBLE_OPT_LL_MAX_PKT_SIZE)
 #endif
 
 #ifndef NIMBLE_OPT_LL_CONN_INIT_MAX_TX_BYTES


[2/2] incubator-mynewt-core git commit: Modify write suggested default data length command in controller

Posted by we...@apache.org.
Modify write suggested default data length command in controller

The prior implementation of this command allowed the host to set
the initial max tx octets/time for a device to a value that it
did not support. The controller will now ignore the host request
if it is not in the supported range.


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

Branch: refs/heads/develop
Commit: 73ee5e8b0b0913280637af866d41c54a633aef25
Parents: b0a8c8e
Author: William San Filippo <wi...@runtime.io>
Authored: Mon Apr 18 23:12:10 2016 -0700
Committer: William San Filippo <wi...@runtime.io>
Committed: Mon Apr 18 23:30:06 2016 -0700

----------------------------------------------------------------------
 net/nimble/controller/src/ble_ll_hci.c | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-core/blob/73ee5e8b/net/nimble/controller/src/ble_ll_hci.c
----------------------------------------------------------------------
diff --git a/net/nimble/controller/src/ble_ll_hci.c b/net/nimble/controller/src/ble_ll_hci.c
index 14a6b52..8149bf4 100644
--- a/net/nimble/controller/src/ble_ll_hci.c
+++ b/net/nimble/controller/src/ble_ll_hci.c
@@ -281,8 +281,14 @@ ble_ll_hci_le_read_bufsize(uint8_t *rspbuf, uint8_t *rsplen)
 
 #ifdef BLE_LL_CFG_FEAT_DATA_LEN_EXT
 /**
- * HCI write suggested default data length command. Returns the controllers
- * initial max tx octet/time.
+ * HCI write suggested default data length command.
+ *
+ * This command is used by the host to change the initial max tx octets/time
+ * for all connections. Note that if the controller does not support the
+ * requested times no error is returned; the controller simply ignores the
+ * request (but remembers what the host requested for the read suggested
+ * default data length command). The spec allows for the controller to
+ * disregard the host.
  *
  * @param rspbuf Pointer to response buffer
  * @param rsplen Length of response buffer
@@ -304,8 +310,12 @@ ble_ll_hci_le_wr_sugg_data_len(uint8_t *cmdbuf)
     if (ble_ll_chk_txrx_octets(tx_oct) && ble_ll_chk_txrx_time(tx_time)) {
         g_ble_ll_conn_params.sugg_tx_octets = (uint8_t)tx_oct;
         g_ble_ll_conn_params.sugg_tx_time = tx_time;
-        g_ble_ll_conn_params.conn_init_max_tx_octets = tx_oct;
-        g_ble_ll_conn_params.conn_init_max_tx_time = tx_time;
+
+        if ((tx_time < g_ble_ll_conn_params.supp_max_tx_time) &&
+            (tx_oct < g_ble_ll_conn_params.supp_max_tx_octets)) {
+            g_ble_ll_conn_params.conn_init_max_tx_octets = tx_oct;
+            g_ble_ll_conn_params.conn_init_max_tx_time = tx_time;
+        }
         rc = BLE_ERR_SUCCESS;
     } else {
         rc = BLE_ERR_INV_HCI_CMD_PARMS;