You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by ry...@apache.org on 2019/06/17 11:06:41 UTC

[mynewt-nimble] branch master updated (ae020da -> 7c17723)

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

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


    from ae020da  Use common config option for periodic syncs count
     new b6b6958  nimble/ll: Move wfr handling to LL context
     new 7c17723  nimble/ll: Simplify aux_data ref counter usage

The 2 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:
 nimble/controller/include/controller/ble_ll_scan.h |   9 +-
 nimble/controller/src/ble_ll_conn.c                |  63 +++--
 nimble/controller/src/ble_ll_scan.c                | 267 ++++++++++-----------
 nimble/controller/src/ble_ll_sched.c               |   6 +-
 4 files changed, 161 insertions(+), 184 deletions(-)


[mynewt-nimble] 01/02: nimble/ll: Move wfr handling to LL context

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

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

commit b6b6958d77f53310b0ad90cd16b13ddd76f37db2
Author: Łukasz Rymanowski <lu...@codecoup.pl>
AuthorDate: Wed Jun 5 14:52:04 2019 +0200

    nimble/ll: Move wfr handling to LL context
    
    With this patch we move handling scan cleaning to LL context in case of
    wfr timeout. This is in order to avoid races on access to scansm data.
---
 nimble/controller/include/controller/ble_ll_scan.h |  1 +
 nimble/controller/src/ble_ll_conn.c                | 12 +----
 nimble/controller/src/ble_ll_scan.c                | 58 ++++++++++++++--------
 3 files changed, 39 insertions(+), 32 deletions(-)

diff --git a/nimble/controller/include/controller/ble_ll_scan.h b/nimble/controller/include/controller/ble_ll_scan.h
index 746963c..45a8ec2 100644
--- a/nimble/controller/include/controller/ble_ll_scan.h
+++ b/nimble/controller/include/controller/ble_ll_scan.h
@@ -143,6 +143,7 @@ struct ble_ll_scan_sm
     struct os_mbuf *scan_req_pdu;
     struct ble_npl_event scan_sched_ev;
     struct hal_timer scan_timer;
+    struct ble_npl_event scan_wfr_ev;
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
     struct hal_timer duration_timer;
diff --git a/nimble/controller/src/ble_ll_conn.c b/nimble/controller/src/ble_ll_conn.c
index 809797c..d37b7ad 100644
--- a/nimble/controller/src/ble_ll_conn.c
+++ b/nimble/controller/src/ble_ll_conn.c
@@ -554,18 +554,10 @@ ble_ll_conn_init_wfr_timer_exp(void)
     }
 
     ble_ll_conn_reset_pending_aux_conn_rsp();
+    connsm->inita_identity_used = 0;
 
     scansm = connsm->scansm;
-    if (scansm && scansm->cur_aux_data) {
-        if (ble_ll_scan_aux_data_unref(scansm->cur_aux_data)) {
-            ble_ll_scan_aux_data_unref(scansm->cur_aux_data);
-        }
-        scansm->cur_aux_data = NULL;
-        STATS_INC(ble_ll_stats, aux_missed_adv);
-        ble_ll_event_send(&scansm->scan_sched_ev);
-    }
-
-    connsm->inita_identity_used = 0;
+    ble_ll_event_send(&scansm->scan_wfr_ev);
 #endif
 }
 /**
diff --git a/nimble/controller/src/ble_ll_scan.c b/nimble/controller/src/ble_ll_scan.c
index c57dd32..78cb1f3 100644
--- a/nimble/controller/src/ble_ll_scan.c
+++ b/nimble/controller/src/ble_ll_scan.c
@@ -1501,6 +1501,39 @@ ble_ll_aux_scan_rsp_failed(void)
 }
 #endif
 
+static void
+ble_ll_scan_wfr_event_cb(struct ble_npl_event *ev)
+{
+    struct ble_ll_scan_sm *scansm = ev->ev.ev_arg;
+
+    if (!scansm->scan_enabled) {
+        return;
+    }
+
+#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
+    if (scansm && scansm->cur_aux_data) {
+        ble_ll_scan_aux_data_unref(scansm->cur_aux_data);
+        scansm->cur_aux_data = NULL;
+        STATS_INC(ble_ll_stats, aux_missed_adv);
+    }
+#endif
+
+    /*
+    * If we timed out waiting for a response, the scan response pending
+    * flag should be set. Deal with scan backoff. Put device back into rx.
+    */
+
+    if (scansm->scan_rsp_pending) {
+        ble_ll_scan_req_backoff(scansm, 0);
+#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
+        ble_ll_aux_scan_rsp_failed();
+#endif
+    }
+
+    ble_ll_scan_chk_resume();
+    ble_phy_restart_rx();
+}
+
 /**
  * Called to process the scanning OS event which was posted to the LL task
  *
@@ -2610,29 +2643,8 @@ ble_ll_scan_wfr_timer_exp(void)
 {
     struct ble_ll_scan_sm *scansm;
 
-    /*
-     * If we timed out waiting for a response, the scan response pending
-     * flag should be set. Deal with scan backoff. Put device back into rx.
-     */
     scansm = &g_ble_ll_scan_sm;
-    if (scansm->scan_rsp_pending) {
-        ble_ll_scan_req_backoff(scansm, 0);
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-        ble_ll_aux_scan_rsp_failed();
-        ble_ll_scan_chk_resume();
-#endif
-    }
-
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    if (scansm->cur_aux_data) {
-        ble_ll_scan_end_adv_evt(scansm->cur_aux_data);
-        scansm->cur_aux_data = NULL;
-        STATS_INC(ble_ll_stats, aux_missed_adv);
-        ble_ll_scan_chk_resume();
-    }
-#endif
-
-    ble_phy_restart_rx();
+    ble_ll_event_send(&scansm->scan_wfr_ev);
 }
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
@@ -3795,6 +3807,8 @@ ble_ll_scan_common_init(void)
                                                                         scansm);
 #endif
 
+    ble_npl_event_init(&scansm->scan_wfr_ev, ble_ll_scan_wfr_event_cb, scansm);
+
     /* Get a scan request mbuf (packet header) and attach to state machine */
     scansm->scan_req_pdu = os_msys_get_pkthdr(BLE_SCAN_LEGACY_MAX_PKT_LEN,
                                               sizeof(struct ble_mbuf_hdr));


[mynewt-nimble] 02/02: nimble/ll: Simplify aux_data ref counter usage

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

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

commit 7c17723dc9171fdf2dec316019b69b392a5d8a99
Author: Łukasz Rymanowski <lu...@codecoup.pl>
AuthorDate: Wed Jun 5 16:47:13 2019 +0200

    nimble/ll: Simplify aux_data ref counter usage
    
    This patch fixes aux_data reference counter, as previosly it was rather
    "usage counter", which was hard to follow. This patch should make it
    easier.
---
 nimble/controller/include/controller/ble_ll_scan.h |   8 +-
 nimble/controller/src/ble_ll_conn.c                |  51 ++---
 nimble/controller/src/ble_ll_scan.c                | 217 +++++++++------------
 nimble/controller/src/ble_ll_sched.c               |   6 +-
 4 files changed, 126 insertions(+), 156 deletions(-)

diff --git a/nimble/controller/include/controller/ble_ll_scan.h b/nimble/controller/include/controller/ble_ll_scan.h
index 45a8ec2..f3906f5 100644
--- a/nimble/controller/include/controller/ble_ll_scan.h
+++ b/nimble/controller/include/controller/ble_ll_scan.h
@@ -92,7 +92,7 @@ struct ble_ll_scan_params
 #define BLE_LL_AUX_INCOMPLETE_BIT       0x02
 #define BLE_LL_AUX_INCOMPLETE_ERR_BIT   0x04
 #define BLE_LL_AUX_HAS_ADDRA            0x08
-#define BLE_LL_AUX_IGNORE_BIT           0x10
+#define BLE_LL_SENT_EVENT_TO_HOST       0x10
 #define BLE_LL_AUX_HAS_DIR_ADDRA        0x20
 #define BLE_LL_AUX_TRUNCATED_SENT       0x40
 #define BLE_LL_AUX_HAS_ADI              0x80
@@ -251,13 +251,13 @@ int ble_ll_scan_parse_ext_hdr(struct os_mbuf *om,
                               struct ble_mbuf_hdr *ble_hdr,
                               struct ble_ll_ext_adv_report *parsed_evt);
 
-void ble_ll_scan_aux_data_ref(struct ble_ll_aux_data *aux_scan);
-int ble_ll_scan_aux_data_unref(struct ble_ll_aux_data *aux_scan);
+struct ble_ll_aux_data *ble_ll_scan_aux_data_ref(struct ble_ll_aux_data *aux_scan);
+void ble_ll_scan_aux_data_unref(struct ble_ll_aux_data *aux_scan);
+void ble_ll_scan_end_adv_evt(struct ble_ll_aux_data *aux_data);
 #endif
 
 /* Called to clean up current aux data */
 void ble_ll_scan_clean_cur_aux_data(void);
-void ble_ll_scan_end_adv_evt(struct ble_ll_aux_data *aux_data);
 
 #ifdef __cplusplus
 }
diff --git a/nimble/controller/src/ble_ll_conn.c b/nimble/controller/src/ble_ll_conn.c
index d37b7ad..8721f1e 100644
--- a/nimble/controller/src/ble_ll_conn.c
+++ b/nimble/controller/src/ble_ll_conn.c
@@ -2678,22 +2678,25 @@ ble_ll_init_rx_pkt_in(uint8_t pdu_type, uint8_t *rxbuf,
     struct ble_ll_conn_sm *connsm;
     int ext_adv_mode = -1;
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    struct ble_ll_aux_data *aux_data = ble_hdr->rxinfo.user_data;
-
-    /*
-     * Let's take the reference for handover to LL.
-     * There shall be one more, if not something went very wrong
-     */
-    if (aux_data && !ble_ll_scan_aux_data_unref(aux_data)) {
-        BLE_LL_ASSERT(0);
-        return;
-    }
+    struct ble_ll_aux_data *aux_data = NULL;
 
+     if (ble_hdr->rxinfo.user_data) {
+         /* aux_data just a local helper, no need to ref
+          * as ble_hdr->rxinfo.user_data is unref in the end of this function
+          */
+         aux_data = ble_hdr->rxinfo.user_data;
+     }
 #endif
 
     /* Get the connection state machine we are trying to create */
     connsm = g_ble_ll_conn_create_sm;
     if (!connsm) {
+#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
+        if (aux_data) {
+            ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data);
+            ble_hdr->rxinfo.user_data = NULL;
+        }
+#endif
         return;
     }
 
@@ -2710,7 +2713,9 @@ ble_ll_init_rx_pkt_in(uint8_t pdu_type, uint8_t *rxbuf,
         if (BLE_MBUF_HDR_WAIT_AUX(ble_hdr)) {
             /* Just continue scanning. We are waiting for AUX */
             if (!ble_ll_sched_aux_scan(ble_hdr, connsm->scansm, aux_data)) {
-                ble_ll_scan_aux_data_ref(aux_data);
+                /* ref for aux ptr in the scheduler */
+                ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data);
+                ble_hdr->rxinfo.user_data = NULL;
                 ble_ll_scan_chk_resume();
                 return;
             }
@@ -2719,8 +2724,9 @@ ble_ll_init_rx_pkt_in(uint8_t pdu_type, uint8_t *rxbuf,
     }
 
     if (CONN_F_AUX_CONN_REQ(connsm)) {
-        /* Wait for connection response */
         if (pdu_type != BLE_ADV_PDU_TYPE_AUX_CONNECT_RSP) {
+            /* Wait for connection response, in this point of time aux is NULL */
+            BLE_LL_ASSERT(ble_hdr->rxinfo.user_data == NULL);
             return;
         }
     }
@@ -2791,7 +2797,10 @@ ble_ll_init_rx_pkt_in(uint8_t pdu_type, uint8_t *rxbuf,
         /* Lets take last used phy */
         ble_ll_conn_init_phy(connsm, ble_hdr->rxinfo.phy);
 #endif
-        ble_ll_scan_aux_data_unref(aux_data);
+        if (aux_data) {
+            ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data);
+            ble_hdr->rxinfo.user_data = NULL;
+        }
 #endif
         ble_ll_conn_created(connsm, NULL);
         return;
@@ -2800,7 +2809,10 @@ ble_ll_init_rx_pkt_in(uint8_t pdu_type, uint8_t *rxbuf,
 scan_continue:
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
     /* Drop last reference and keep continue to connect */
-    ble_ll_scan_aux_data_unref(aux_data);
+    if (aux_data) {
+        ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data);
+        ble_hdr->rxinfo.user_data = NULL;
+    }
 #endif
     ble_ll_scan_chk_resume();
 }
@@ -2955,10 +2967,6 @@ ble_ll_init_rx_isr_end(uint8_t *rxbuf, uint8_t crcok,
     if (scansm->cur_aux_data) {
         ble_hdr->rxinfo.user_data = scansm->cur_aux_data;
         scansm->cur_aux_data = NULL;
-        if (ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data) == 0) {
-            ble_hdr->rxinfo.user_data = 0;
-            goto init_rx_isr_exit;
-        }
     }
 #endif
 
@@ -3000,8 +3008,6 @@ ble_ll_init_rx_isr_end(uint8_t *rxbuf, uint8_t crcok,
         if (rc < 0) {
             /* No memory or broken packet */
             ble_hdr->rxinfo.flags |= BLE_MBUF_HDR_F_AUX_INVALID;
-            ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data);
-            ble_hdr->rxinfo.user_data = NULL;
             goto init_rx_isr_exit;
         }
     }
@@ -3215,11 +3221,6 @@ ble_ll_init_rx_isr_end(uint8_t *rxbuf, uint8_t crcok,
 
 init_rx_isr_exit:
 
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    if (ble_hdr->rxinfo.user_data) {
-        ble_ll_scan_aux_data_ref(ble_hdr->rxinfo.user_data);
-    }
-#endif
     /*
      * We have to restart receive if we cant hand up pdu. We return 0 so that
      * the phy does not get disabled.
diff --git a/nimble/controller/src/ble_ll_scan.c b/nimble/controller/src/ble_ll_scan.c
index 78cb1f3..c30e6b5 100644
--- a/nimble/controller/src/ble_ll_scan.c
+++ b/nimble/controller/src/ble_ll_scan.c
@@ -176,12 +176,14 @@ ble_ll_aux_scan_cb(struct ble_ll_sched_item *sch)
      */
     if (!scansm->scan_enabled || scansm->cur_aux_data) {
         ble_ll_scan_aux_data_unref(sch->cb_arg);
+        sch->cb_arg = NULL;
         goto done;
     }
 
     /* Check if there is no aux connect sent. If so drop the sched item */
     if (lls == BLE_LL_STATE_INITIATING && ble_ll_conn_init_pending_aux_conn_rsp()) {
         ble_ll_scan_aux_data_unref(sch->cb_arg);
+        sch->cb_arg = NULL;
         goto done;
     }
 
@@ -199,13 +201,12 @@ ble_ll_aux_scan_cb(struct ble_ll_sched_item *sch)
 
     /* When doing RX for AUX pkt, cur_aux_data keeps valid aux data */
     scansm->cur_aux_data = sch->cb_arg;
+    sch->cb_arg = NULL;
     BLE_LL_ASSERT(scansm->cur_aux_data != NULL);
     scansm->cur_aux_data->scanning = 1;
 
     if (ble_ll_scan_start(scansm, sch)) {
-        ble_ll_scan_aux_data_unref(scansm->cur_aux_data);
-        scansm->cur_aux_data = NULL;
-        ble_ll_scan_chk_resume();
+        ble_ll_event_send(&scansm->scan_wfr_ev);
         goto done;
     }
 
@@ -460,14 +461,14 @@ ble_ll_scan_send_truncated_if_chained(struct ble_ll_aux_data *aux_data)
     struct ble_ll_ext_adv_report *evt;
 
     if (!ble_ll_hci_is_le_event_enabled(BLE_HCI_LE_SUBEV_EXT_ADV_RPT)) {
-        goto done;
+        return;
     }
 
     BLE_LL_ASSERT(aux_data);
 
     if (!BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_CHAIN_BIT)) {
         /* if not chained, there is nothing to do here */
-        goto done;
+        return;
     }
 
     if (aux_data->evt) {
@@ -476,7 +477,7 @@ ble_ll_scan_send_truncated_if_chained(struct ble_ll_aux_data *aux_data)
     } else {
         evt = ble_ll_scan_init_ext_adv_report(NULL);
         if (!evt) {
-            goto done;
+            return;
         }
     }
 
@@ -497,9 +498,6 @@ ble_ll_scan_send_truncated_if_chained(struct ble_ll_aux_data *aux_data)
 
     evt->sid = (aux_data->adi >> 12);
     ble_ll_hci_event_send((uint8_t *)evt);
-
-done:
-    ble_ll_scan_aux_data_unref(aux_data);
 }
 
 static int
@@ -519,29 +517,19 @@ ble_ll_scan_get_adi(struct ble_ll_aux_data *aux_data, uint16_t *adi)
 }
 #endif
 
+#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
 void
 ble_ll_scan_end_adv_evt(struct ble_ll_aux_data *aux_data)
 {
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    /*
-     * Check if tuncated has been sent
-     *
-     * Normally reference counter here should be 2. 1 for outstanding
-     * complete event and one for ongoing scanning. If there is only 1
-     * that means, advertising event is already completed
-     * (truncated was sent to the host) and we just need to drop last reference.
-     * Otherwise we should try to send truncated event to the host.
+    /* If part of the event has been sent to the host and truncated
+     * has not been sent, do it now
      */
-    if (!BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_TRUNCATED_SENT)) {
+    if (BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_SENT_EVENT_TO_HOST) &&
+            !BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_TRUNCATED_SENT)) {
         ble_ll_scan_send_truncated_if_chained(aux_data);
     }
-
-    if (ble_ll_scan_aux_data_unref(aux_data) > 0) {
-        BLE_LL_ASSERT(0);
-    }
-
-#endif
 }
+#endif
 /**
  * Do scan machine clean up on PHY disabled
  *
@@ -555,6 +543,7 @@ ble_ll_scan_clean_cur_aux_data(void)
     /* If scanner was reading aux ptr, we need to clean it up */
     if (scansm && scansm->cur_aux_data) {
         ble_ll_scan_end_adv_evt(scansm->cur_aux_data);
+        ble_ll_scan_aux_data_unref(scansm->cur_aux_data);
         scansm->cur_aux_data = NULL;
     }
 #endif
@@ -1278,35 +1267,31 @@ ble_ll_scan_aux_data_free(struct ble_ll_aux_data *aux_scan)
     }
 }
 
-void
+struct ble_ll_aux_data *
 ble_ll_scan_aux_data_ref(struct ble_ll_aux_data *aux_data)
 {
     os_sr_t sr;
 
-    if (!aux_data) {
-        return;
-    }
+    BLE_LL_ASSERT(aux_data);
 
     OS_ENTER_CRITICAL(sr);
     aux_data->ref_cnt++;
     ble_ll_trace_u32x2(BLE_LL_TRACE_ID_AUX_REF, (uint32_t) aux_data, aux_data->ref_cnt);
 
     OS_EXIT_CRITICAL(sr);
+
+    return aux_data;
 }
 
-int
+void
 ble_ll_scan_aux_data_unref(struct ble_ll_aux_data *aux_data)
 {
     os_sr_t sr;
-    int ret;
 
-    if (!aux_data) {
-        return 0;
-    }
+    BLE_LL_ASSERT(aux_data);
 
     OS_ENTER_CRITICAL(sr);
     aux_data->ref_cnt--;
-    ret = aux_data->ref_cnt;
     ble_ll_trace_u32x2(BLE_LL_TRACE_ID_AUX_UNREF, (uint32_t) aux_data, aux_data->ref_cnt);
 
     if (aux_data->ref_cnt == 0) {
@@ -1314,14 +1299,12 @@ ble_ll_scan_aux_data_unref(struct ble_ll_aux_data *aux_data)
     }
 
     OS_EXIT_CRITICAL(sr);
-
-    return ret;
 }
 
 static void
 ble_ll_scan_sched_remove(struct ble_ll_sched_item *sch)
 {
-    ble_ll_scan_aux_data_free(sch->cb_arg);
+    ble_ll_scan_aux_data_unref(sch->cb_arg);
 }
 #endif
 /**
@@ -1504,7 +1487,7 @@ ble_ll_aux_scan_rsp_failed(void)
 static void
 ble_ll_scan_wfr_event_cb(struct ble_npl_event *ev)
 {
-    struct ble_ll_scan_sm *scansm = ev->ev.ev_arg;
+    struct ble_ll_scan_sm *scansm = &g_ble_ll_scan_sm;
 
     if (!scansm->scan_enabled) {
         return;
@@ -1512,6 +1495,10 @@ ble_ll_scan_wfr_event_cb(struct ble_npl_event *ev)
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
     if (scansm && scansm->cur_aux_data) {
+        if (scansm->scan_rsp_pending) {
+            STATS_INC(ble_ll_stats, aux_scan_rsp_err);
+        }
+        ble_ll_scan_end_adv_evt(scansm->cur_aux_data);
         ble_ll_scan_aux_data_unref(scansm->cur_aux_data);
         scansm->cur_aux_data = NULL;
         STATS_INC(ble_ll_stats, aux_missed_adv);
@@ -1525,9 +1512,6 @@ ble_ll_scan_wfr_event_cb(struct ble_npl_event *ev)
 
     if (scansm->scan_rsp_pending) {
         ble_ll_scan_req_backoff(scansm, 0);
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-        ble_ll_aux_scan_rsp_failed();
-#endif
     }
 
     ble_ll_scan_chk_resume();
@@ -1960,12 +1944,12 @@ ble_ll_scan_get_aux_data(struct ble_mbuf_hdr *ble_hdr, uint8_t *rxbuf)
             }
 
             BLE_LL_AUX_SET_FLAG(new_aux, BLE_LL_AUX_CHAIN_BIT);
-            BLE_LL_AUX_SET_FLAG(new_aux, BLE_LL_AUX_INCOMPLETE_BIT);
         } else {
             if (ble_ll_scan_ext_adv_init(&new_aux) < 0) {
             /* Out of memory */
             return -1;
             }
+            BLE_LL_AUX_SET_FLAG(new_aux, BLE_LL_AUX_INCOMPLETE_BIT);
         }
 
         new_aux->aux_phy = tmp_aux_data.aux_phy;
@@ -2327,13 +2311,7 @@ ble_ll_scan_rx_isr_end(struct os_mbuf *rxpdu, uint8_t crcok)
      * return 0 in this case because we dont want the phy disabled.
      */
     if (rxpdu == NULL) {
-        if (scansm->scan_rsp_pending) {
-            ble_ll_scan_req_backoff(scansm, 0);
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-            ble_ll_aux_scan_rsp_failed();
-#endif
-        }
-        ble_phy_restart_rx();
+        ble_ll_event_send(&scansm->scan_wfr_ev);
         return 0;
     }
 
@@ -2349,12 +2327,6 @@ ble_ll_scan_rx_isr_end(struct os_mbuf *rxpdu, uint8_t crcok)
     if (scansm->cur_aux_data) {
         ble_hdr->rxinfo.user_data = scansm->cur_aux_data;
         scansm->cur_aux_data = NULL;
-        if (ble_ll_scan_aux_data_unref(ble_hdr->rxinfo.user_data) == 0) {
-            ble_hdr->rxinfo.user_data = NULL;
-            ble_hdr->rxinfo.flags |= BLE_MBUF_HDR_F_AUX_INVALID;
-            goto scan_rx_isr_exit;
-        }
-
         /* If we were expecting aux/chain and it not arrived,
          * lets just exit here.
          */
@@ -2366,9 +2338,6 @@ ble_ll_scan_rx_isr_end(struct os_mbuf *rxpdu, uint8_t crcok)
 
     /* Just leave if the CRC is not OK. */
     if (!crcok) {
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-        ble_hdr->rxinfo.flags |= BLE_MBUF_HDR_F_AUX_INVALID;
-#endif
         goto scan_rx_isr_exit;
     }
 
@@ -2557,8 +2526,8 @@ ble_ll_scan_rx_isr_end(struct os_mbuf *rxpdu, uint8_t crcok)
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
                 if (ble_hdr->rxinfo.channel <  BLE_PHY_NUM_DATA_CHANS) {
                     /* Let's keep the aux ptr as a reference to scan rsp */
-                    scansm->cur_aux_data = ble_hdr->rxinfo.user_data;
-                    ble_ll_scan_aux_data_ref(scansm->cur_aux_data);
+                    scansm->cur_aux_data =
+                            ble_ll_scan_aux_data_ref(ble_hdr->rxinfo.user_data);
                     STATS_INC(ble_ll_stats, aux_scan_req_tx);
                 }
 #endif
@@ -2570,12 +2539,6 @@ scan_rx_isr_exit:
     if (rc) {
         ble_ll_state_set(BLE_LL_STATE_STANDBY);
     }
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    /* On handover lets increase ref count */
-    if (ble_hdr->rxinfo.user_data) {
-        ble_ll_scan_aux_data_ref(ble_hdr->rxinfo.user_data);
-    }
-#endif
     return rc;
 }
 
@@ -2757,10 +2720,16 @@ ble_ll_hci_send_ext_adv_report(uint8_t ptype, uint8_t *adva, uint8_t adva_type,
                 evt->evt_type |= (BLE_HCI_ADV_DATA_STATUS_TRUNCATED);
                 BLE_LL_AUX_SET_FLAG(aux_data, BLE_LL_AUX_TRUNCATED_SENT);
                 rc = -1;
+                if (!BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_SENT_EVENT_TO_HOST)) {
+                    ble_hci_trans_buf_free((uint8_t *)evt);
+                    goto done;
+                }
+
             }
         } else if (aux_data ) {
             if (BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_INCOMPLETE_ERR_BIT)) {
                 evt->evt_type |= (BLE_HCI_ADV_DATA_STATUS_TRUNCATED);
+                BLE_LL_AUX_SET_FLAG(aux_data, BLE_LL_AUX_TRUNCATED_SENT);
                 rc = -1;
             } else if (BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_INCOMPLETE_BIT)) {
                 evt->evt_type |= (BLE_HCI_ADV_DATA_STATUS_INCOMPLETE);
@@ -2773,6 +2742,10 @@ ble_ll_hci_send_ext_adv_report(uint8_t ptype, uint8_t *adva, uint8_t adva_type,
 
         ble_ll_hci_event_send((uint8_t *)evt);
 
+        if (aux_data) {
+            BLE_LL_AUX_SET_FLAG(aux_data, BLE_LL_SENT_EVENT_TO_HOST);
+        }
+
         evt = next_evt;
     } while ((offset < datalen) && evt);
 
@@ -2787,15 +2760,23 @@ ble_ll_hci_send_ext_adv_report(uint8_t ptype, uint8_t *adva, uint8_t adva_type,
     }
 
 done:
-    /* If advertising event is completed or failed, we can drop the reference */
-    if (rc <= 0) {
-        if (aux_data){
-            if ((rc == 0) && (aux_data->evt_type & BLE_HCI_ADV_SCAN_RSP_MASK)) {
-                /* Scan response completed successfully */
-                ble_ll_scan_add_scan_rsp_adv(aux_data->addr, aux_data->addr_type,
-                                             1, aux_data->adi);
-            }
-            ble_ll_scan_aux_data_unref(aux_data);
+    /* Incomplete event. Can leave now.*/
+    if (rc == 1) {
+        return rc;
+    }
+
+    if (aux_data){
+        if ((rc == 0) && (aux_data->evt_type & BLE_HCI_ADV_SCAN_RSP_MASK)) {
+            /* Scan response completed successfully, add to duplicate list
+             * if possible
+             */
+            ble_ll_scan_add_scan_rsp_adv(aux_data->addr, aux_data->addr_type,
+                                         1, aux_data->adi);
+        }
+
+        /* Error during sending event or even before.*/
+        if (rc < 0) {
+            BLE_LL_AUX_SET_FLAG(aux_data, BLE_LL_AUX_INCOMPLETE_ERR_BIT);
         }
     }
 
@@ -2888,9 +2869,15 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
     struct ble_mbuf_hdr *ble_hdr;
     int ext_adv_mode = -1;
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    struct ble_ll_aux_data *aux_data = hdr->rxinfo.user_data;
+    struct ble_ll_aux_data *aux_data;
     int rc;
     uint8_t evt_possibly_truncated = 0;
+
+    /* No need to ref as this is only local helper
+     * and unref on hdr->rxinfo.user_data is done in the end of this function
+     */
+    aux_data = hdr->rxinfo.user_data;
+
 #endif
 
     /* Set scan response check flag */
@@ -2898,16 +2885,17 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
 
     /* We dont care about scan requests or connect requests */
     if (!BLE_MBUF_HDR_CRC_OK(hdr)) {
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-        if (BLE_MBUF_HDR_AUX_INVALID(hdr)) {
-            evt_possibly_truncated = 1;
-        }
-#endif
         goto scan_continue;
     }
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    if (aux_data && ptype != BLE_ADV_PDU_TYPE_ADV_EXT_IND) {
+    if (BLE_MBUF_HDR_AUX_INVALID(hdr)) {
+        evt_possibly_truncated = 1;
+        goto scan_continue;
+    }
+
+    if (aux_data && (ptype != BLE_ADV_PDU_TYPE_ADV_EXT_IND ||
+                     BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_INCOMPLETE_ERR_BIT))) {
         /* LL was scheduled for aux but received something different.
          * Let's just ignore received event and send truncated for
          * previous report if needed.
@@ -2971,9 +2959,6 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
     if (ble_ll_scan_chk_filter_policy(ptype, adv_addr, txadd, init_addr,
                                       init_addr_type,
                                       BLE_MBUF_HDR_DEVMATCH(hdr))) {
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-        ble_ll_scan_aux_data_unref(aux_data);
-#endif
         goto scan_continue;
     }
 
@@ -3014,11 +2999,6 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
     /* Filter duplicates */
     if (scansm->scan_filt_dups) {
         if (ble_ll_scan_is_dup_adv(hdr, ptype, ident_addr_type, ident_addr)) {
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-            if (ptype == BLE_ADV_PDU_TYPE_ADV_EXT_IND) {
-                ble_ll_scan_aux_data_unref(aux_data);
-            }
-#endif
             goto scan_continue;
         }
     }
@@ -3026,25 +3006,15 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
     if (ptype == BLE_ADV_PDU_TYPE_ADV_EXT_IND) {
         if (!scansm->ext_scanning) {
-            ble_ll_scan_aux_data_unref(aux_data);
-            goto scan_continue;
-        }
-
-        if (BLE_MBUF_HDR_AUX_INVALID(hdr)) {
-            evt_possibly_truncated = 1;
-            goto scan_continue;
-        }
-
-        /* If it is ignore it means event is already truncated, just unref aux */
-        if (aux_data && BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_IGNORE_BIT)) {
-            ble_ll_scan_aux_data_unref(aux_data);
             goto scan_continue;
         }
 
         /* Let's see if that packet contains aux ptr*/
         if (BLE_MBUF_HDR_WAIT_AUX(hdr)) {
             BLE_LL_ASSERT(aux_data);
-            if (ble_ll_sched_aux_scan(hdr, scansm, hdr->rxinfo.user_data)) {
+
+            /* ble_ll_sched_aux_scan will ref aux_data */
+            if (ble_ll_sched_aux_scan(hdr, scansm, aux_data)) {
                 /* We are here when could not schedule the aux ptr */
                 hdr->rxinfo.flags &= ~BLE_MBUF_HDR_F_AUX_PTR_WAIT;
                 /* Mark that chain is trimmed */
@@ -3052,28 +3022,22 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
                 /* Note: aux_data unref will be done when truncated is sent to the host or
                  * below if we failed to schedule for the very first aux packet.
                  */
-            } else {
-                /* We are here because successfully scheduled for next aux */
-                ble_ll_scan_aux_data_ref(aux_data);
+                if (!BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_SENT_EVENT_TO_HOST)) {
+                    goto scan_continue;
+                }
             }
 
             /*
              * If this is ext adv, there is nothing to do here but just leave and wait
-             * for aux packet. However, if we was not able to schedule for first aux packet,
-             * make sure to unref aux_data here
+             * for aux packet.
              */
             if (!BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_CHAIN_BIT)) {
-                if (BLE_LL_AUX_CHECK_FLAG(aux_data, BLE_LL_AUX_INCOMPLETE_ERR_BIT)) {
-                    ble_ll_scan_aux_data_unref(aux_data);
-                }
                 goto scan_continue;
             }
 
             STATS_INC(ble_ll_stats, aux_chain_cnt);
         }
 
-        /* For the time when sending events up, lets increase ref count */
-        ble_ll_scan_aux_data_ref(aux_data);
         rc = ble_ll_hci_send_ext_adv_report(ptype, ident_addr, ident_addr_type,
                                             init_addr, init_addr_type, om, hdr);
         if (rc < 0) {
@@ -3085,26 +3049,27 @@ ble_ll_scan_rx_pkt_in(uint8_t ptype, struct os_mbuf *om, struct ble_mbuf_hdr *hd
             if (BLE_MBUF_HDR_WAIT_AUX(hdr)) {
                 hdr->rxinfo.flags &= ~BLE_MBUF_HDR_F_AUX_PTR_WAIT;
                 if (ble_ll_sched_rmv_elem(&aux_data->sch) == 0) {
-                    ble_ll_scan_aux_data_unref(aux_data);
+                    /* AUX PTR removed from the scheduler.
+                     * Unref aux_data which are stored in the scheduler item
+                     * as a cb_arg
+                     */
+                    ble_ll_scan_aux_data_unref(aux_data->sch.cb_arg);
+                    aux_data->sch.cb_arg = NULL;
+                    evt_possibly_truncated = 1;
                 }
             }
-            BLE_LL_AUX_SET_FLAG(aux_data, BLE_LL_AUX_IGNORE_BIT);
         } else if ((rc == 0) && scansm->scan_filt_dups && aux_data && aux_data->adi) {
                 ble_ll_scan_add_dup_adv(ident_addr, ident_addr_type,
                                         BLE_HCI_LE_SUBEV_EXT_ADV_RPT,
                                         aux_data->evt_type, 1, aux_data->adi);
         }
-        ble_ll_scan_aux_data_unref(aux_data);
-
         ble_ll_scan_switch_phy(scansm);
 
         if (scansm->scan_rsp_pending) {
             if (!scan_rsp_chk) {
-                /* We are here because we sent SCAN_REQ and wait for SCAN_RSP.
-                 * We do not drop reference here by purpose, because
-                 * it was already dropped in ble_ll_hci_send_ext_adv_report() as
-                 * very first advertising report for scannable report is "completed"
-                 */
+                /* We are here because we sent SCAN_REQ and wait for SCAN_RSP. */
+                ble_ll_scan_aux_data_unref(hdr->rxinfo.user_data);
+                hdr->rxinfo.user_data = NULL;
                 return;
             }
 
@@ -3131,10 +3096,10 @@ scan_continue:
     if (aux_data) {
         if (evt_possibly_truncated) {
             ble_ll_scan_end_adv_evt(aux_data);
-        } else {
-            /* This is ref for handover to LL */
-            ble_ll_scan_aux_data_unref(aux_data);
         }
+
+        ble_ll_scan_aux_data_unref(hdr->rxinfo.user_data);
+        hdr->rxinfo.user_data = NULL;
     }
 #endif
     /*
diff --git a/nimble/controller/src/ble_ll_sched.c b/nimble/controller/src/ble_ll_sched.c
index d0a8f56..caf477b 100644
--- a/nimble/controller/src/ble_ll_sched.c
+++ b/nimble/controller/src/ble_ll_sched.c
@@ -1761,6 +1761,11 @@ ble_ll_sched_aux_scan(struct ble_mbuf_hdr *ble_hdr,
 
 done:
 
+    if (rc == 0) {
+        sch->cb_arg = ble_ll_scan_aux_data_ref(aux_scan);
+        STATS_INC(ble_ll_stats, aux_scheduled);
+    }
+
     /* Get head of list to restart timer */
 #ifdef BLE_XCVR_RFCLK
     entry = TAILQ_FIRST(&g_ble_ll_sched_q);
@@ -1779,7 +1784,6 @@ done:
     BLE_LL_ASSERT(sch != NULL);
     os_cputime_timer_start(&g_ble_ll_sched_timer, sch->start_time);
 
-    STATS_INC(ble_ll_stats, aux_scheduled);
     return rc;
 }
 #endif