You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by an...@apache.org on 2019/12/17 15:57:50 UTC

[mynewt-nimble] 06/09: nimble/ll: Refactor rfclk management from LL

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

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

commit 491a37e31fc364308a64841bb01bbb3a581b1044
Author: Andrzej Kaczmarek <an...@codecoup.pl>
AuthorDate: Fri Dec 6 14:47:32 2019 +0100

    nimble/ll: Refactor rfclk management from LL
    
    Recently we found multiple issues due to invalid rfclk management, i.e.
    either rfclk was enabled when it should be disabled or the other way
    around. The reason for this is rfclk is triggered at various places due
    to either items in scheduler or scannning, but there is no single state
    which can be used to calculate required rfclk state at any given time.
    And this is also more and more complex with all the new cool features we
    added recently.
    
    So now we have such state: ble_ll_rfmgmt (referred to as rfmgmt from now
    on). This is basically ble_ll_xcvr rewritten from scratch. The name was
    changed since in theory this can be used not only to enable RF clock,
    but RF in general - it's up to PHY what it actually does.
    
    rfmgmt is the only place where RF can be enabled (with one exception -
    more on this later) or disabled. This decision is made based on state
    updates from scanner and scheduler:
    - each time scanner moves to new scan window, rfmgmt needs to be notified
      on its start time ,
    - each time 1st item in scheduler is changed, rfmgmt needs to be notified
      on current start time
    - each time there is done-like event which ends conn/adv/whatever event,
      rfmgmt needs to be notified that RF is no longer needed
    
    This basically means:
    - ble_ll_rfmgmt_scan_changed() shall be only called from ble_ll_scan,
    - ble_ll_rfmgmt_sched_changed() shall be only called from ble_ll_sched,
    - ble_ll_rfmgmt_release() can be called whenever we don't RF anymore.
    
    RF is not disabled immediately after calling release, this is postponed
    to LL task so any scan or scheduler update done after release will still
    keep RF running if necessary - all this means that it's safe to simply
    call release at the beginning of function and then do other stuff.
    
    One exception mentioned earlier is when we need to enable RF and this is
    not triggered from other RF activity. Currently this applies to 1st scan
    window, 1st advertising PDU or 1st PDU in DTM. In these cases we just
    basically want to start "now" so it's done as follows:
    - call ble_ll_rfmgmt_enable_now() to enable RF instantly, if disabled
    - this returns tick at which RF is fully enabled (can be "now" if RF is
      already enabled) and this is also the time which should be used to
      start new activity
    
    And last but not least: we do not need "#ifdef BLE_LL_RFCLK" everywhere
    in code anymore!
---
 nimble/controller/include/controller/ble_ll.h      |  30 --
 .../controller/include/controller/ble_ll_rfmgmt.h  |  63 ++++
 .../controller/include/controller/ble_ll_sched.h   |   5 -
 nimble/controller/include/controller/ble_ll_xcvr.h |  48 ---
 nimble/controller/src/ble_ll.c                     |  34 +-
 nimble/controller/src/ble_ll_adv.c                 |  80 ++---
 nimble/controller/src/ble_ll_conn.c                |  12 +-
 nimble/controller/src/ble_ll_dtm.c                 |  20 +-
 nimble/controller/src/ble_ll_rfmgmt.c              | 346 +++++++++++++++++++++
 nimble/controller/src/ble_ll_scan.c                | 177 +++--------
 nimble/controller/src/ble_ll_sched.c               | 134 ++------
 nimble/controller/src/ble_ll_sync.c                |  21 +-
 nimble/controller/src/ble_ll_xcvr.c                | 169 ----------
 nimble/drivers/nrf51/src/ble_phy.c                 |  11 -
 nimble/drivers/nrf52/src/ble_phy.c                 |  11 -
 15 files changed, 519 insertions(+), 642 deletions(-)

diff --git a/nimble/controller/include/controller/ble_ll.h b/nimble/controller/include/controller/ble_ll.h
index a07f8c8..b748e10 100644
--- a/nimble/controller/include/controller/ble_ll.h
+++ b/nimble/controller/include/controller/ble_ll.h
@@ -56,27 +56,6 @@ extern "C" {
 #else
 #define BLE_LL_ASSERT(cond) assert(cond)
 #endif
-/*
- * XXX:
- * I guess this should not depend on the 32768 crystal to be honest. This
- * should be done for TIMER0 as well since the rf clock chews up more current.
- * Deal with this later.
- *
- * Another note: BLE_XTAL_SETTLE_TIME should be bsp related (I guess). There
- * should be a note in there that the converted usecs to ticks value of this
- * should not be 0. Thus: if you are using a 32.768 os cputime freq, the min
- * value of settle time should be 31 usecs. I would suspect all settling times
- * would exceed 31 usecs.
- */
-
-/* Determines if we need to turn on/off rf clock */
-#undef BLE_XCVR_RFCLK
-
-/* We will turn on/off rf clock */
-#if MYNEWT_VAL(BLE_XTAL_SETTLE_TIME) != 0
-#define BLE_XCVR_RFCLK
-
-#endif
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LE_2M_PHY) || MYNEWT_VAL(BLE_LL_CFG_FEAT_LE_CODED_PHY)
 #define BLE_LL_BT5_PHY_SUPPORTED    (1)
@@ -116,15 +95,6 @@ struct ble_ll_obj
     uint8_t ll_pref_tx_phys;
     uint8_t ll_pref_rx_phys;
 
-#ifdef BLE_XCVR_RFCLK
-    uint8_t ll_rfclk_state;
-    uint8_t ll_rfclk_is_sched;
-    uint16_t ll_xtal_ticks;
-    uint32_t ll_rfclk_start_time;
-    uint32_t ll_rfclk_sched_time;
-    struct hal_timer ll_rfclk_timer;
-#endif
-
     /* Task event queue */
     struct ble_npl_eventq ll_evq;
 
diff --git a/nimble/controller/include/controller/ble_ll_rfmgmt.h b/nimble/controller/include/controller/ble_ll_rfmgmt.h
new file mode 100644
index 0000000..da584e5
--- /dev/null
+++ b/nimble/controller/include/controller/ble_ll_rfmgmt.h
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef H_BLE_LL_RFMGMT_
+#define H_BLE_LL_RFMGMT_
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+void ble_ll_rfmgmt_init(void);
+
+#if MYNEWT_VAL(BLE_XTAL_SETTLE_TIME) > 0
+
+void ble_ll_rfmgmt_reset(void);
+
+/* Notify rfmgmt that scan window has changed (only called from ble_ll_scan) */
+void ble_ll_rfmgmt_scan_changed(bool enabled, uint32_t next_window);
+
+/* Notify rfmgmt that 1st scheduled item has changed (only called from ble_ll_sched) */
+void ble_ll_rfmgmt_sched_changed(struct ble_ll_sched_item *first);
+
+/* Notify rfmgmt that RF is no longer needed by current event */
+void ble_ll_rfmgmt_release(void);
+
+/* Enables RF immediately and returns tick at which RF will be fully enabled */
+uint32_t ble_ll_rfmgmt_enable_now(void);
+
+/* Returns true only if RF is currently fully enabled (i.e. not off or enabling) */
+bool ble_ll_rfmgmt_is_enabled(void);
+
+#else
+
+static inline void ble_ll_rfmgmt_reset(void) { }
+static inline void ble_ll_rfmgmt_scan_changed(bool e, uint32_t n) { }
+static inline void ble_ll_rfmgmt_sched_changed(struct ble_ll_sched_item *f) { }
+static inline void ble_ll_rfmgmt_release(void) { }
+static inline uint32_t ble_ll_rfmgmt_enable_now(void) { return 0; }
+static inline bool ble_ll_rfmgmt_is_enabled(void) { return true; }
+
+#endif
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* H_BLE_LL_RFMGMT_ */
diff --git a/nimble/controller/include/controller/ble_ll_sched.h b/nimble/controller/include/controller/ble_ll_sched.h
index 9c638e5..72d5c73 100644
--- a/nimble/controller/include/controller/ble_ll_sched.h
+++ b/nimble/controller/include/controller/ble_ll_sched.h
@@ -210,11 +210,6 @@ int ble_ll_sched_scan_req_over_aux_ptr(uint32_t chan, uint8_t phy_mode);
 /* Stop the scheduler */
 void ble_ll_sched_stop(void);
 
-#ifdef BLE_XCVR_RFCLK
-/* Check if RF clock needs to be restarted */
-void ble_ll_sched_rfclk_chk_restart(void);
-#endif
-
 #if MYNEWT_VAL(BLE_LL_DTM)
 int ble_ll_sched_dtm(struct ble_ll_sched_item *sch);
 #endif
diff --git a/nimble/controller/include/controller/ble_ll_xcvr.h b/nimble/controller/include/controller/ble_ll_xcvr.h
deleted file mode 100644
index a7ab0e8..0000000
--- a/nimble/controller/include/controller/ble_ll_xcvr.h
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef H_BLE_LL_XCVR_
-#define H_BLE_LL_XCVR_
-
-#ifdef __cplusplus
-extern "C" {
-#endif
-
-#ifdef BLE_XCVR_RFCLK
-
-/* RF clock states */
-#define BLE_RFCLK_STATE_OFF     (0)
-#define BLE_RFCLK_STATE_ON      (1)
-#define BLE_RFCLK_STATE_SETTLED (2)
-
-int ble_ll_xcvr_rfclk_state(void);
-void ble_ll_xcvr_rfclk_start_now(void);
-void ble_ll_xcvr_rfclk_stop(void);
-void ble_ll_xcvr_rfclk_enable(void);
-void ble_ll_xcvr_rfclk_disable(void);
-uint32_t ble_ll_xcvr_rfclk_time_till_settled(void);
-void ble_ll_xcvr_rfclk_timer_exp(void *arg);
-void ble_ll_xcvr_rfclk_timer_start(uint32_t cputime);
-#endif
-
-#ifdef __cplusplus
-}
-#endif
-
-#endif /* H_LL_ */
diff --git a/nimble/controller/src/ble_ll.c b/nimble/controller/src/ble_ll.c
index 68fffda..3ab918b 100644
--- a/nimble/controller/src/ble_ll.c
+++ b/nimble/controller/src/ble_ll.c
@@ -40,7 +40,7 @@
 #include "controller/ble_ll_hci.h"
 #include "controller/ble_ll_whitelist.h"
 #include "controller/ble_ll_resolv.h"
-#include "controller/ble_ll_xcvr.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "controller/ble_ll_trace.h"
 #include "controller/ble_ll_sync.h"
 #include "ble_ll_conn_priv.h"
@@ -1326,16 +1326,12 @@ ble_ll_reset(void)
     int rc;
     os_sr_t sr;
 
-    /* Stop the phy */
-    ble_phy_disable();
-
-    /* Stop any wait for response timer */
     OS_ENTER_CRITICAL(sr);
+    ble_phy_disable();
     ble_ll_sched_stop();
-    OS_EXIT_CRITICAL(sr);
-
-    /* Stop any scanning */
     ble_ll_scan_reset();
+    ble_ll_rfmgmt_reset();
+    OS_EXIT_CRITICAL(sr);
 
     /* Stop any advertising */
     ble_ll_adv_reset();
@@ -1372,13 +1368,6 @@ ble_ll_reset(void)
     /* Set state to standby */
     ble_ll_state_set(BLE_LL_STATE_STANDBY);
 
-#ifdef BLE_XCVR_RFCLK
-    OS_ENTER_CRITICAL(sr);
-    /* Stops rf clock and rfclock timer */
-    ble_ll_xcvr_rfclk_stop();
-    OS_EXIT_CRITICAL(sr);
-#endif
-
     /* Reset our random address */
     memset(g_random_addr, 0, BLE_DEV_ADDR_LEN);
 
@@ -1508,9 +1497,6 @@ ble_ll_init(void)
 {
     int rc;
     uint32_t features;
-#ifdef BLE_XCVR_RFCLK
-    uint32_t xtal_ticks;
-#endif
     ble_addr_t addr;
     struct ble_ll_obj *lldata;
 
@@ -1533,17 +1519,7 @@ ble_ll_init(void)
         }
     }
 
-#ifdef BLE_XCVR_RFCLK
-    /* Settling time of crystal, in ticks */
-    xtal_ticks = MYNEWT_VAL(BLE_XTAL_SETTLE_TIME);
-    BLE_LL_ASSERT(xtal_ticks != 0);
-    g_ble_ll_data.ll_xtal_ticks = ble_ll_usecs_to_ticks_round_up(xtal_ticks);
-
-    /* Initialize rf clock timer */
-    os_cputime_timer_init(&g_ble_ll_data.ll_rfclk_timer,
-                          ble_ll_xcvr_rfclk_timer_exp, NULL);
-
-#endif
+    ble_ll_rfmgmt_init();
 
     /* Get pointer to global data object */
     lldata = &g_ble_ll_data;
diff --git a/nimble/controller/src/ble_ll_adv.c b/nimble/controller/src/ble_ll_adv.c
index dd98609..22cd559 100644
--- a/nimble/controller/src/ble_ll_adv.c
+++ b/nimble/controller/src/ble_ll_adv.c
@@ -39,7 +39,7 @@
 #include "controller/ble_ll_resolv.h"
 #include "controller/ble_ll_trace.h"
 #include "controller/ble_ll_utils.h"
-#include "controller/ble_ll_xcvr.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "ble_ll_conn_priv.h"
 
 /* XXX: TODO
@@ -1853,6 +1853,8 @@ ble_ll_adv_sm_stop(struct ble_ll_adv_sm *advsm)
     os_sr_t sr;
 
     if (advsm->adv_enabled) {
+        ble_ll_rfmgmt_release();
+
         /* Remove any scheduled advertising items */
         ble_ll_sched_rmv_elem(&advsm->adv_sch);
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
@@ -1879,9 +1881,6 @@ ble_ll_adv_sm_stop(struct ble_ll_adv_sm *advsm)
             ble_ll_scan_chk_resume();
         }
 #endif
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
         OS_EXIT_CRITICAL(sr);
 
         ble_npl_eventq_remove(&g_ble_ll_data.ll_evq, &advsm->adv_txdone_ev);
@@ -2413,6 +2412,8 @@ ble_ll_adv_periodic_done(struct ble_ll_adv_sm *advsm)
     assert(advsm->periodic_adv_active);
     assert(advsm->periodic_sync_active);
 
+    ble_ll_rfmgmt_release();
+
     sync = SYNC_CURRENT(advsm);
     sync_next = SYNC_NEXT(advsm);
 
@@ -2430,11 +2431,6 @@ ble_ll_adv_periodic_done(struct ble_ll_adv_sm *advsm)
     /* Check if we need to resume scanning */
     ble_ll_scan_chk_resume();
 
-    /* Turn off the clock if not doing anything else */
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_sched_rfclk_chk_restart();
-#endif
-
     advsm->periodic_sync_active = 0;
     ble_ll_adv_update_periodic_data(advsm);
     ble_ll_adv_reschedule_periodic_event(advsm);
@@ -2502,6 +2498,8 @@ ble_ll_adv_sm_stop_periodic(struct ble_ll_adv_sm *advsm)
 {
     os_sr_t sr;
 
+    ble_ll_rfmgmt_release();
+
     if (!advsm->periodic_adv_active) {
         return;
     }
@@ -2533,10 +2531,6 @@ ble_ll_adv_sm_stop_periodic(struct ble_ll_adv_sm *advsm)
         g_ble_ll_cur_adv_sm = NULL;
         ble_ll_scan_chk_resume();
     }
-
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_sched_rfclk_chk_restart();
-#endif
     OS_EXIT_CRITICAL(sr);
 
     ble_ll_adv_flags_clear(advsm, BLE_LL_ADV_SM_FLAG_PERIODIC_SYNC_SENDING);
@@ -2569,11 +2563,9 @@ ble_ll_adv_sm_start(struct ble_ll_adv_sm *advsm)
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LE_CSA2)
     uint32_t access_addr;
 #endif
-#ifdef BLE_XCVR_RFCLK
-    int xtal_state;
-    os_sr_t sr;
-#endif
     const uint8_t *random_addr;
+    uint32_t earliest_start_time;
+    int32_t delta;
 
     /* only clear flags that are not set from HCI */
     ble_ll_adv_flags_clear(advsm, BLE_LL_ADV_SM_FLAG_TX_ADD |
@@ -2656,32 +2648,31 @@ ble_ll_adv_sm_start(struct ble_ll_adv_sm *advsm)
     advsm->adv_chan = adv_chan;
 
     /*
-     * Set initial start time (randomized similar to interval)
-     * NOTE: adv_event_start_time gets set by the sched_adv_new
+     * Scheduling 1st PDU is a bit tricky.
+     * Earliest possible start time is after RF is enabled so just force RF to
+     * start here to see when if will be fully enabled - it will be too early,
+     * but this is the only reliable way to have it enabled on time.
+     * Next we calculate expected start time (randomize it a bit) and this is
+     * used to setup start time for scheduler item.
+     * Then we check if start time for scheduler item (which includes scheduler
+     * overhead) is no earlier than calculated earliest possible start time and
+     * adjust scheduler item if necessary.
      */
+    earliest_start_time = ble_ll_rfmgmt_enable_now();
+
     start_delay_us = rand() % (BLE_LL_ADV_DELAY_MS_MAX * 1000);
     advsm->adv_pdu_start_time = os_cputime_get32() +
                                 os_cputime_usecs_to_ticks(start_delay_us);
 
-#ifdef BLE_XCVR_RFCLK
-    OS_ENTER_CRITICAL(sr);
-    xtal_state = ble_ll_xcvr_rfclk_state();
-    if (xtal_state != BLE_RFCLK_STATE_SETTLED) {
-        if (xtal_state == BLE_RFCLK_STATE_OFF) {
-            advsm->adv_pdu_start_time += g_ble_ll_data.ll_xtal_ticks;
-            ble_ll_xcvr_rfclk_start_now();
-        } else {
-            advsm->adv_pdu_start_time += ble_ll_xcvr_rfclk_time_till_settled();
-        }
+    ble_ll_adv_set_sched(advsm);
+
+    delta = (int32_t)(advsm->adv_sch.start_time - earliest_start_time);
+    if (delta < 0) {
+        advsm->adv_sch.start_time -= delta;
+        advsm->adv_sch.end_time -= delta;
     }
-    OS_EXIT_CRITICAL(sr);
-#endif
 
-    /*
-     * Schedule advertising. We set the initial schedule start and end
-     * times to the earliest possible start/end.
-     */
-    ble_ll_adv_set_sched(advsm);
+    /* This does actual scheduling */
     ble_ll_sched_adv_new(&advsm->adv_sch, ble_ll_adv_scheduled, NULL);
 
     /* we start periodic before AE since we need PDU start time in SyncInfo */
@@ -4473,6 +4464,8 @@ ble_ll_adv_done(struct ble_ll_adv_sm *advsm)
 
     assert(advsm->adv_enabled);
 
+    ble_ll_rfmgmt_release();
+
     ble_ll_adv_update_adv_scan_rsp_data(advsm);
 
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
@@ -4504,14 +4497,8 @@ ble_ll_adv_done(struct ble_ll_adv_sm *advsm)
         }
 #endif
 
-        /* Check if we need to resume scanning */
         ble_ll_scan_chk_resume();
 
-        /* Turn off the clock if not doing anything else */
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
-
         /* This event is over. Set adv channel to first one */
         advsm->adv_chan = ble_ll_adv_first_chan(advsm);
 
@@ -4663,6 +4650,9 @@ ble_ll_adv_sec_done(struct ble_ll_adv_sm *advsm)
     aux = AUX_CURRENT(advsm);
     aux_next = AUX_NEXT(advsm);
 
+    /* We don't need RF anymore */
+    ble_ll_rfmgmt_release();
+
     if (advsm->aux_not_scanned) {
         ble_ll_sched_rmv_elem(&aux_next->sch);
     }
@@ -4685,14 +4675,8 @@ ble_ll_adv_sec_done(struct ble_ll_adv_sm *advsm)
         return;
     }
 
-    /* Check if we need to resume scanning */
     ble_ll_scan_chk_resume();
 
-    /* Turn off the clock if not doing anything else */
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_sched_rfclk_chk_restart();
-#endif
-
     /* Check if advertising timed out */
     if (advsm->duration && (advsm->adv_pdu_start_time >= advsm->adv_end_time)) {
         ble_ll_adv_sm_stop_timeout(advsm);
diff --git a/nimble/controller/src/ble_ll_conn.c b/nimble/controller/src/ble_ll_conn.c
index ffb3995..98fe9b7 100644
--- a/nimble/controller/src/ble_ll_conn.c
+++ b/nimble/controller/src/ble_ll_conn.c
@@ -38,6 +38,7 @@
 #include "controller/ble_ll_resolv.h"
 #include "controller/ble_ll_adv.h"
 #include "controller/ble_ll_trace.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "controller/ble_phy.h"
 #include "controller/ble_hw.h"
 #include "controller/ble_ll_utils.h"
@@ -2367,6 +2368,8 @@ ble_ll_conn_event_end(struct ble_npl_event *ev)
     uint32_t tmo;
     struct ble_ll_conn_sm *connsm;
 
+    ble_ll_rfmgmt_release();
+
     /* Better be a connection state machine! */
     connsm = (struct ble_ll_conn_sm *)ble_npl_event_get_arg(ev);
     BLE_LL_ASSERT(connsm);
@@ -2382,10 +2385,6 @@ ble_ll_conn_event_end(struct ble_npl_event *ev)
         ble_ll_state_set(BLE_LL_STATE_STANDBY);
 
         ble_ll_scan_chk_resume();
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
-
         return;
     }
 
@@ -2393,13 +2392,8 @@ ble_ll_conn_event_end(struct ble_npl_event *ev)
     ble_ll_trace_u32x2(BLE_LL_TRACE_ID_CONN_EV_END, connsm->conn_handle,
                        connsm->event_cntr);
 
-    /* Check if we need to resume scanning */
     ble_ll_scan_chk_resume();
 
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_sched_rfclk_chk_restart();
-#endif
-
     /* If we have transmitted the terminate IND successfully, we are done */
     if ((connsm->csmflags.cfbit.terminate_ind_txd) ||
                     (connsm->csmflags.cfbit.terminate_ind_rxd &&
diff --git a/nimble/controller/src/ble_ll_dtm.c b/nimble/controller/src/ble_ll_dtm.c
index 0b8314b..fa284e0 100644
--- a/nimble/controller/src/ble_ll_dtm.c
+++ b/nimble/controller/src/ble_ll_dtm.c
@@ -28,7 +28,7 @@
 #include "controller/ble_ll.h"
 #include "controller/ble_phy.h"
 #include "controller/ble_ll_sched.h"
-#include "controller/ble_ll_xcvr.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "ble_ll_dtm_priv.h"
 
 STATS_SECT_START(ble_ll_dtm_stats)
@@ -368,8 +368,7 @@ schedule:
     sch->sched_cb = ble_ll_dtm_tx_sched_cb;
     sch->cb_arg = &g_ble_ll_dtm_ctx;
     sch->sched_type = BLE_LL_SCHED_TYPE_DTM;
-    sch->start_time =  os_cputime_get32() +
-                                       os_cputime_usecs_to_ticks(5000);
+    sch->start_time =  ble_ll_rfmgmt_enable_now();
 
     /* Prepare os_event */
     ble_npl_event_init(&g_ble_ll_dtm_ctx.evt, ble_ll_dtm_ev_tx_resched_cb,
@@ -412,14 +411,6 @@ ble_ll_dtm_rx_start(void)
 
     ble_ll_state_set(BLE_LL_STATE_DTM);
 
-#ifdef BLE_XCVR_RFCLK
-    OS_ENTER_CRITICAL(sr);
-    if (ble_ll_xcvr_rfclk_state() == BLE_RFCLK_STATE_OFF) {
-        ble_ll_xcvr_rfclk_start_now();
-    }
-    OS_EXIT_CRITICAL(sr);
-#endif
-
     return 0;
 }
 
@@ -451,8 +442,7 @@ ble_ll_dtm_rx_create_ctx(uint8_t rf_channel, uint8_t phy_mode)
     sch->sched_cb = ble_ll_dtm_rx_sched_cb;
     sch->cb_arg = &g_ble_ll_dtm_ctx;
     sch->sched_type = BLE_LL_SCHED_TYPE_DTM;
-    sch->start_time =  os_cputime_get32() +
-                                       os_cputime_usecs_to_ticks(5000);
+    sch->start_time =  ble_ll_rfmgmt_enable_now();
 
     rc = ble_ll_sched_dtm(sch);
     BLE_LL_ASSERT(rc == 0);
@@ -480,9 +470,7 @@ ble_ll_dtm_ctx_free(struct dtm_ctx * ctx)
     ble_phy_disable();
     ble_phy_disable_dtm();
     ble_ll_state_set(BLE_LL_STATE_STANDBY);
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_xcvr_rfclk_stop();
-#endif
+    ble_ll_rfmgmt_release();
 
     os_mbuf_free_chain(ctx->om);
     memset(ctx, 0, sizeof(*ctx));
diff --git a/nimble/controller/src/ble_ll_rfmgmt.c b/nimble/controller/src/ble_ll_rfmgmt.c
new file mode 100644
index 0000000..58dad7c
--- /dev/null
+++ b/nimble/controller/src/ble_ll_rfmgmt.c
@@ -0,0 +1,346 @@
+/*
+ * 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 <stddef.h>
+#include <assert.h>
+#include <stddef.h>
+#include "syscfg/syscfg.h"
+#include "os/os_cputime.h"
+#include "controller/ble_phy.h"
+#include "controller/ble_ll.h"
+#include "controller/ble_ll_sched.h"
+#include "controller/ble_ll_rfmgmt.h"
+
+#if MYNEWT_VAL(BLE_XTAL_SETTLE_TIME) > 0
+
+enum ble_ll_rfmgmt_state {
+    RFMGMT_STATE_OFF = 0,
+    RFMGMT_STATE_ENABLING = 1,
+    RFMGMT_STATE_ENABLED = 2,
+};
+
+struct ble_ll_rfmgmt_data {
+    enum ble_ll_rfmgmt_state state;
+    uint16_t ticks_to_enabled;
+
+    struct hal_timer timer;
+    bool timer_scheduled;
+    uint32_t timer_scheduled_at;
+
+    bool enable_scan;
+    bool enable_sched;
+    uint32_t enable_scan_at;
+    uint32_t enable_sched_at;
+
+    uint32_t enabled_at;
+
+    struct ble_npl_event release_ev;
+};
+
+static struct ble_ll_rfmgmt_data g_ble_ll_rfmgmt_data;
+
+static void
+ble_ll_rfmgmt_enable(void)
+{
+    OS_ASSERT_CRITICAL();
+
+    if (g_ble_ll_rfmgmt_data.state == RFMGMT_STATE_OFF) {
+        g_ble_ll_rfmgmt_data.state = RFMGMT_STATE_ENABLING;
+        g_ble_ll_rfmgmt_data.enabled_at = os_cputime_get32();
+        ble_phy_rfclk_enable();
+    }
+}
+
+static void
+ble_ll_rfmgmt_disable(void)
+{
+    OS_ASSERT_CRITICAL();
+
+    if (g_ble_ll_rfmgmt_data.state != RFMGMT_STATE_OFF) {
+        ble_phy_rfclk_disable();
+        g_ble_ll_rfmgmt_data.state = RFMGMT_STATE_OFF;
+    }
+}
+
+static void
+ble_ll_rfmgmt_timer_reschedule(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    uint32_t enable_at;
+
+    /* Figure out when we need to enable RF */
+    if (rfmgmt->enable_scan && rfmgmt->enable_sched) {
+        if (CPUTIME_LT(rfmgmt->enable_scan_at, rfmgmt->enable_sched_at)) {
+            enable_at = rfmgmt->enable_scan_at;
+        } else {
+            enable_at = rfmgmt->enable_sched_at;
+        }
+    } else if (rfmgmt->enable_scan) {
+        enable_at = rfmgmt->enable_scan_at;
+    } else if (rfmgmt->enable_sched) {
+        enable_at = rfmgmt->enable_sched_at;
+    } else {
+        rfmgmt->timer_scheduled = false;
+        os_cputime_timer_stop(&rfmgmt->timer);
+        return;
+    }
+
+    if (rfmgmt->timer_scheduled) {
+        /*
+         * If there is timer already scheduled at the same time we do not need
+         * to do anything. Otherwise we need to stop timer and schedule it again
+         * regardless if it's earlier or later to make sure it fires at the time
+         * something expects it.
+         */
+
+        if (rfmgmt->timer_scheduled_at == enable_at) {
+            return;
+        }
+
+        rfmgmt->timer_scheduled = false;
+        os_cputime_timer_stop(&rfmgmt->timer);
+    }
+
+    /*
+     * In case timer was requested to be enabled before current time, just make
+     * sure it's enabled and assume caller can deal with this. This will happen
+     * if something is scheduled "now" since "enable_at" is in the past, but in
+     * such case it's absolutely harmless since we already have clock enabled
+     * and this will do nothing.
+     */
+    if (CPUTIME_LEQ(enable_at, os_cputime_get32())) {
+        ble_ll_rfmgmt_enable();
+        return;
+    }
+
+    rfmgmt->timer_scheduled = true;
+    rfmgmt->timer_scheduled_at = enable_at;
+    os_cputime_timer_start(&rfmgmt->timer, enable_at);
+}
+
+static void
+ble_ll_rfmgmt_timer_exp(void *arg)
+{
+    g_ble_ll_rfmgmt_data.timer_scheduled = false;
+    ble_ll_rfmgmt_enable();
+}
+
+static void
+ble_ll_rfmgmt_release_ev(struct ble_npl_event *ev)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    uint32_t now;
+    bool can_disable;
+    uint8_t lls;
+    os_sr_t sr;
+
+    OS_ENTER_CRITICAL(sr);
+
+    now = os_cputime_get32();
+
+    can_disable = true;
+    lls = ble_ll_state_get();
+
+    if (rfmgmt->enable_scan && CPUTIME_GEQ(now, rfmgmt->enable_scan_at)) {
+        /* Blocked by scan */
+        can_disable = false;
+    } else if (rfmgmt->enable_sched && CPUTIME_GEQ(now, rfmgmt->enable_sched_at)) {
+        /* Blocked by scheduler item */
+        can_disable = false;
+    } else if (lls != BLE_LL_STATE_STANDBY) {
+        /* Blocked by LL state */
+        can_disable = false;
+    }
+
+    if (can_disable) {
+        ble_ll_rfmgmt_disable();
+    }
+
+    OS_EXIT_CRITICAL(sr);
+}
+
+static uint32_t
+ble_ll_rfmgmt_ticks_to_enabled(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    uint32_t rem_ticks;
+    uint32_t now;
+
+    switch (rfmgmt->state) {
+    case RFMGMT_STATE_OFF:
+        rem_ticks = rfmgmt->ticks_to_enabled;
+        break;
+    case RFMGMT_STATE_ENABLING:
+        now = os_cputime_get32();
+        if (CPUTIME_LT(now, rfmgmt->enabled_at + rfmgmt->ticks_to_enabled)) {
+            rem_ticks = rfmgmt->enabled_at + rfmgmt->ticks_to_enabled - now;
+            break;
+        }
+        rfmgmt->state = RFMGMT_STATE_ENABLED;
+        /* no break */
+    case RFMGMT_STATE_ENABLED:
+        rem_ticks = 0;
+        break;
+    default:
+        BLE_LL_ASSERT(0);
+        rem_ticks = 0;
+        break;
+    }
+
+    return rem_ticks;
+}
+
+void
+ble_ll_rfmgmt_init(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+
+    rfmgmt->state = RFMGMT_STATE_OFF;
+
+    rfmgmt->ticks_to_enabled =
+            ble_ll_usecs_to_ticks_round_up(MYNEWT_VAL(BLE_XTAL_SETTLE_TIME));
+
+    rfmgmt->timer_scheduled = false;
+    os_cputime_timer_init(&rfmgmt->timer, ble_ll_rfmgmt_timer_exp, NULL);
+
+    ble_npl_event_init(&rfmgmt->release_ev, ble_ll_rfmgmt_release_ev, NULL);
+}
+
+void
+ble_ll_rfmgmt_reset(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+
+    rfmgmt->timer_scheduled = false;
+    rfmgmt->timer_scheduled_at = 0;
+    os_cputime_timer_stop(&rfmgmt->timer);
+
+    ble_npl_eventq_remove(&g_ble_ll_data.ll_evq, &rfmgmt->release_ev);
+
+    ble_ll_rfmgmt_disable();
+
+    rfmgmt->enable_scan = false;
+    rfmgmt->enable_scan_at = 0;
+    rfmgmt->enable_sched = false;
+    rfmgmt->enable_sched_at = 0;
+
+    rfmgmt->enabled_at = 0;
+}
+
+void
+ble_ll_rfmgmt_scan_changed(bool enabled, uint32_t next_window)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    os_sr_t sr;
+
+    OS_ENTER_CRITICAL(sr);
+
+    rfmgmt->enable_scan = enabled;
+    rfmgmt->enable_scan_at = next_window - rfmgmt->ticks_to_enabled;
+
+    ble_ll_rfmgmt_timer_reschedule();
+
+    OS_EXIT_CRITICAL(sr);
+}
+
+void
+ble_ll_rfmgmt_sched_changed(struct ble_ll_sched_item *first)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    os_sr_t sr;
+
+    OS_ENTER_CRITICAL(sr);
+
+    rfmgmt->enable_sched = (first != NULL);
+    if (first) {
+        rfmgmt->enable_sched_at = first->start_time - rfmgmt->ticks_to_enabled;
+    }
+
+    ble_ll_rfmgmt_timer_reschedule();
+
+    OS_EXIT_CRITICAL(sr);
+}
+
+void
+ble_ll_rfmgmt_release(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    os_sr_t sr;
+
+    OS_ENTER_CRITICAL(sr);
+
+    ble_npl_eventq_remove(&g_ble_ll_data.ll_evq, &rfmgmt->release_ev);
+
+    if (g_ble_ll_rfmgmt_data.state != RFMGMT_STATE_OFF) {
+        ble_npl_eventq_put(&g_ble_ll_data.ll_evq, &rfmgmt->release_ev);
+    }
+
+    OS_EXIT_CRITICAL(sr);
+}
+
+uint32_t
+ble_ll_rfmgmt_enable_now(void)
+{
+    struct ble_ll_rfmgmt_data *rfmgmt = &g_ble_ll_rfmgmt_data;
+    uint32_t enabled_at;
+    os_sr_t sr;
+
+    OS_ENTER_CRITICAL(sr);
+
+    ble_ll_rfmgmt_enable();
+
+    if (rfmgmt->state == RFMGMT_STATE_ENABLED) {
+        enabled_at = os_cputime_get32();
+    } else {
+        enabled_at = rfmgmt->enabled_at + rfmgmt->ticks_to_enabled + 1;
+    }
+
+    OS_EXIT_CRITICAL(sr);
+
+    return enabled_at;
+}
+
+bool
+ble_ll_rfmgmt_is_enabled(void)
+{
+    bool ret;
+
+    OS_ASSERT_CRITICAL();
+
+    ret = ble_ll_rfmgmt_ticks_to_enabled() == 0;
+
+    return ret;
+}
+
+#else
+
+void
+ble_ll_rfmgmt_init(void)
+{
+    static bool enabled = false;
+
+    if (!enabled) {
+        ble_phy_rfclk_enable();
+    }
+
+    enabled = true;
+}
+
+#endif
diff --git a/nimble/controller/src/ble_ll_scan.c b/nimble/controller/src/ble_ll_scan.c
index 456553f..f707ebf 100644
--- a/nimble/controller/src/ble_ll_scan.c
+++ b/nimble/controller/src/ble_ll_scan.c
@@ -37,7 +37,7 @@
 #include "controller/ble_ll_hci.h"
 #include "controller/ble_ll_whitelist.h"
 #include "controller/ble_ll_resolv.h"
-#include "controller/ble_ll_xcvr.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "controller/ble_ll_trace.h"
 #include "controller/ble_ll_sync.h"
 #include "ble_ll_conn_priv.h"
@@ -1083,36 +1083,6 @@ ble_ll_scan_start(struct ble_ll_scan_sm *scansm, struct ble_ll_sched_item *sch)
     return rc;
 }
 
-#ifdef BLE_XCVR_RFCLK
-static void
-ble_ll_scan_rfclk_chk_stop(void)
-{
-    int stop;
-    int32_t time_till_next;
-    os_sr_t sr;
-    uint32_t next_time;
-
-    stop = 0;
-    OS_ENTER_CRITICAL(sr);
-    if (ble_ll_sched_next_time(&next_time)) {
-        /*
-         * If the time until the next event is too close, dont bother to turn
-         * off the clock
-         */
-        time_till_next = (int32_t)(next_time - os_cputime_get32());
-        if (time_till_next > g_ble_ll_data.ll_xtal_ticks) {
-            stop = 1;
-        }
-    } else {
-        stop = 1;
-    }
-    if (stop) {
-        ble_ll_xcvr_rfclk_disable();
-    }
-    OS_EXIT_CRITICAL(sr);
-}
-#endif
-
 static uint8_t
 ble_ll_scan_get_next_adv_prim_chan(uint8_t chan)
 {
@@ -1148,7 +1118,6 @@ static bool
 ble_ll_scan_is_inside_window(struct ble_ll_scan_params *scanp, uint32_t time)
 {
     uint32_t start_time;
-    uint32_t dt;
 
     /* Make sure we are checking against closest window */
     start_time = ble_ll_scan_move_window_to(scanp, time);
@@ -1158,19 +1127,10 @@ ble_ll_scan_is_inside_window(struct ble_ll_scan_params *scanp, uint32_t time)
         return true;
     }
 
-    if (CPUTIME_LT(time, start_time) ||
-        CPUTIME_GEQ(time, start_time + scanp->timing.window)) {
-#ifdef BLE_XCVR_RFCLK
-        dt = time - start_time;
-        if (dt < (scanp->timing.interval - g_ble_ll_data.ll_xtal_ticks)) {
-            ble_ll_scan_rfclk_chk_stop();
-        }
-#endif
-        return false;
-    }
-
-    return true;
+    return CPUTIME_GEQ(time, start_time) &&
+           CPUTIME_LT(time, start_time + scanp->timing.window);
 }
+
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
 static void
 ble_ll_scan_aux_data_free(struct ble_ll_aux_data *aux_data)
@@ -1256,6 +1216,7 @@ ble_ll_scan_sm_stop(int chk_disable)
     os_cputime_timer_stop(&scansm->scan_timer);
 
     OS_ENTER_CRITICAL(sr);
+
     /* Disable scanning state machine */
     scansm->scan_enabled = 0;
     scansm->restart_timer_needed = 0;
@@ -1264,6 +1225,7 @@ ble_ll_scan_sm_stop(int chk_disable)
     if (scansm->ext_scanning) {
         ble_ll_scan_clean_cur_aux_data();
         ble_ll_sched_rmv_elem_type(BLE_LL_SCHED_TYPE_AUX_SCAN, ble_ll_scan_sched_remove);
+        scansm->ext_scanning = 0;
     }
 #endif
 
@@ -1289,23 +1251,15 @@ ble_ll_scan_sm_stop(int chk_disable)
 
             /* Set LL state to standby */
             ble_ll_state_set(BLE_LL_STATE_STANDBY);
-
-            /* May need to stop the rfclk */
-#ifdef BLE_XCVR_RFCLK
-            ble_ll_scan_rfclk_chk_stop();
-#endif
         }
         OS_EXIT_CRITICAL(sr);
     }
 
-#if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
-    if (scansm->ext_scanning) {
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
-        scansm->ext_scanning = 0;
-    }
-#endif
+    /* No need for RF anymore */
+    OS_ENTER_CRITICAL(sr);
+    ble_ll_rfmgmt_scan_changed(false, 0);
+    ble_ll_rfmgmt_release();
+    OS_EXIT_CRITICAL(sr);
 }
 
 static int
@@ -1345,9 +1299,13 @@ ble_ll_scan_sm_start(struct ble_ll_scan_sm *scansm)
     os_mempool_clear(&g_scan_dup_pool);
     TAILQ_INIT(&g_scan_dup_list);
 
-    /* XXX: align to current or next slot???. */
-    /* Schedule start time now */
-    scanp->timing.start_time = os_cputime_get32();
+    /*
+     * First scan window can start when RF is enabled. Add 1 tick since we are
+     * most likely not aligned with ticks so RF may be effectively enabled 1
+     * tick later.
+     */
+    scanp->timing.start_time = ble_ll_rfmgmt_enable_now();
+    ble_ll_rfmgmt_scan_changed(true, scanp->timing.start_time);
 
     if (scanp_next) {
         /* Schedule start time right after first phy */
@@ -1355,8 +1313,8 @@ ble_ll_scan_sm_start(struct ble_ll_scan_sm *scansm)
                                         scanp->timing.window;
     }
 
-    /* Post scanning event to start off the scanning process */
-    ble_ll_event_send(&scansm->scan_sched_ev);
+    /* Start scan at 1st window */
+    os_cputime_timer_start(&scansm->scan_timer, scanp->timing.start_time);
 
     return BLE_ERR_SUCCESS;
 }
@@ -1425,7 +1383,7 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
 {
     struct ble_ll_scan_sm *scansm;
     os_sr_t sr;
-    int start_scan;
+    bool start_scan;
     bool inside_window;
     struct ble_ll_scan_params *scanp;
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
@@ -1434,11 +1392,6 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
 #endif
     uint32_t next_proc_time;
     uint32_t now;
-#ifdef BLE_XCVR_RFCLK
-    uint32_t xtal_ticks;
-    int xtal_state;
-#endif
-
     /*
      * Get the scanning state machine. If not enabled (this is possible), just
      * leave and do nothing (just make sure timer is stopped).
@@ -1452,6 +1405,8 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
     OS_ENTER_CRITICAL(sr);
     if (!scansm->scan_enabled) {
         os_cputime_timer_stop(&scansm->scan_timer);
+        ble_ll_rfmgmt_scan_changed(false, 0);
+        ble_ll_rfmgmt_release();
         OS_EXIT_CRITICAL(sr);
         return;
     }
@@ -1491,17 +1446,21 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
 
     /*
      * At this point scanp and scanp_next point to current or closest scan
-     * window on both PHYs (scanp is the closer one).
-     *
-     * Now we need to calculate when next scan proc should happen. If we are
-     * outside window, fire at the start of closest window to start scan. If we
-     * are inside window, fire at the end of window to either disable scan or
-     * switch to next PHY.
+     * window on both PHYs (scanp is the closer one). Make sure RF is enabled
+     * on time.
      */
-    if (!inside_window) {
-        next_proc_time = scanp->timing.start_time;
-    } else {
+    ble_ll_rfmgmt_scan_changed(true, scanp->timing.start_time);
+
+    /*
+     * If we are inside window, next scan proc should happen at the end of
+     * current window to either disable scan or switch to next PHY.
+     * If we are outside window, next scan proc should happen at the time of
+     * closest scan window.
+     */
+    if (inside_window) {
         next_proc_time = scanp->timing.start_time + scanp->timing.window;
+    } else {
+        next_proc_time = scanp->timing.start_time;
     }
 
     /*
@@ -1509,12 +1468,12 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
      * scanning event was overlapped in the schedule. In this case all we do
      * is post the scan schedule end event.
      */
-    start_scan = 1;
+    start_scan = inside_window;
     switch (ble_ll_state_get()) {
     case BLE_LL_STATE_ADV:
     case BLE_LL_STATE_CONNECTION:
     case BLE_LL_STATE_SYNC:
-         start_scan = 0;
+         start_scan = false;
          break;
     case BLE_LL_STATE_INITIATING:
         /* Must disable PHY since we will move to a new channel */
@@ -1539,66 +1498,12 @@ ble_ll_scan_event_proc(struct ble_npl_event *ev)
         break;
     }
 
-    if (start_scan && inside_window) {
-#ifdef BLE_XCVR_RFCLK
-        xtal_state = ble_ll_xcvr_rfclk_state();
-        if (xtal_state != BLE_RFCLK_STATE_SETTLED) {
-            if (xtal_state == BLE_RFCLK_STATE_OFF) {
-                xtal_ticks = g_ble_ll_data.ll_xtal_ticks;
-            } else {
-                xtal_ticks = ble_ll_xcvr_rfclk_time_till_settled();
-            }
-
-            /*
-             * Only bother if we have enough time to receive anything
-             * here. The next event time will turn off the clock.
-             */
-            if (scanp->timing.start_time + scanp->timing.window - now <=
-                                                                xtal_ticks) {
-                goto done;
-            }
-
-            /*
-             * If clock off, start clock. Set next event time to now plus
-             * the clock setting time.
-             */
-            if (xtal_state == BLE_RFCLK_STATE_OFF) {
-                ble_ll_xcvr_rfclk_start_now();
-            }
-            next_proc_time = now + xtal_ticks;
-            goto done;
-        }
-#endif
+    if (start_scan) {
         ble_ll_scan_start(scansm, NULL);
-        goto done;
-    }
-
-#ifdef BLE_XCVR_RFCLK
-    if (!inside_window) {
-        /*
-         * We need to wake up before we need to start scanning in order
-         * to make sure the rfclock is on. If we are close to being on,
-         * enable the rfclock. If not, set wakeup time.
-         */
-        if (scanp->timing.start_time - now  <= g_ble_ll_data.ll_xtal_ticks) {
-            /* Start the clock if necessary */
-            if (start_scan) {
-                if (ble_ll_xcvr_rfclk_state() == BLE_RFCLK_STATE_OFF) {
-                    ble_ll_xcvr_rfclk_start_now();
-                    next_proc_time = now + g_ble_ll_data.ll_xtal_ticks;
-                }
-            }
-        } else {
-            next_proc_time = scanp->timing.start_time -
-                             g_ble_ll_data.ll_xtal_ticks;
-            if (start_scan) {
-                ble_ll_scan_rfclk_chk_stop();
-            }
-        }
+    } else {
+        ble_ll_rfmgmt_release();
     }
-#endif
 
-done:
     OS_EXIT_CRITICAL(sr);
     os_cputime_timer_start(&scansm->scan_timer, next_proc_time);
 }
diff --git a/nimble/controller/src/ble_ll_sched.c b/nimble/controller/src/ble_ll_sched.c
index 40a75e8..eef016b 100644
--- a/nimble/controller/src/ble_ll_sched.c
+++ b/nimble/controller/src/ble_ll_sched.c
@@ -28,7 +28,7 @@
 #include "controller/ble_ll_sched.h"
 #include "controller/ble_ll_adv.h"
 #include "controller/ble_ll_scan.h"
-#include "controller/ble_ll_xcvr.h"
+#include "controller/ble_ll_rfmgmt.h"
 #include "controller/ble_ll_trace.h"
 #include "controller/ble_ll_sync.h"
 #include "ble_ll_conn_priv.h"
@@ -36,11 +36,6 @@
 /* XXX: this is temporary. Not sure what I want to do here */
 struct hal_timer g_ble_ll_sched_timer;
 
-#ifdef BLE_XCVR_RFCLK
-/* Settling time of crystal, in ticks */
-uint8_t g_ble_ll_sched_xtal_ticks;
-#endif
-
 uint8_t g_ble_ll_sched_offset_ticks;
 
 #define BLE_LL_SCHED_ADV_WORST_CASE_USECS       \
@@ -274,17 +269,12 @@ ble_ll_sched_conn_reschedule(struct ble_ll_conn_sm *connsm)
         entry = start_overlap;
     }
 
-#ifdef BLE_XCVR_RFCLK
     entry = TAILQ_FIRST(&g_ble_ll_sched_q);
     if (entry == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     } else {
         sch = entry;
     }
-#else
-    /* Get first on list */
-    sch = TAILQ_FIRST(&g_ble_ll_sched_q);
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -532,6 +522,7 @@ ble_ll_sched_master_new(struct ble_ll_conn_sm *connsm,
 
     /* Get head of list to restart timer */
     sch = TAILQ_FIRST(&g_ble_ll_sched_q);
+    ble_ll_rfmgmt_sched_changed(sch);
 
     OS_EXIT_CRITICAL(sr);
 
@@ -714,6 +705,7 @@ ble_ll_sched_master_new(struct ble_ll_conn_sm *connsm,
 
     /* Get head of list to restart timer */
     sch = TAILQ_FIRST(&g_ble_ll_sched_q);
+    ble_ll_rfmgmt_sched_changed(sch);
 
     OS_EXIT_CRITICAL(sr);
 
@@ -740,11 +732,7 @@ ble_ll_sched_slave_new(struct ble_ll_conn_sm *connsm)
     struct ble_ll_sched_item *entry;
     struct ble_ll_sched_item *next_sch;
     struct ble_ll_sched_item *sch;
-
-#ifdef BLE_XCVR_RFCLK
-    int first;
-    first = 0;
-#endif
+    int first = 0;
 
     /* Get schedule element from connection */
     rc = -1;
@@ -775,9 +763,7 @@ ble_ll_sched_slave_new(struct ble_ll_conn_sm *connsm)
     if (!entry) {
         /* Nothing in schedule. Schedule as soon as possible */
         rc = 0;
-#ifdef BLE_XCVR_RFCLK
         first = 1;
-#endif
     } else {
         os_cputime_timer_stop(&g_ble_ll_sched_timer);
         while (1) {
@@ -810,23 +796,18 @@ ble_ll_sched_slave_new(struct ble_ll_conn_sm *connsm)
         if (!rc) {
             sch->enqueued = 1;
         }
-#ifdef BLE_XCVR_RFCLK
+
         next_sch = TAILQ_FIRST(&g_ble_ll_sched_q);
         if (next_sch == sch) {
             first = 1;
         } else {
             sch = next_sch;
         }
-#else
-        sch = TAILQ_FIRST(&g_ble_ll_sched_q);
-#endif
     }
 
-#ifdef BLE_XCVR_RFCLK
     if (first) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     }
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -941,17 +922,12 @@ ble_ll_sched_sync_reschedule(struct ble_ll_sched_item *sch,
         sch->enqueued = 1;
     }
 
-#ifdef BLE_XCVR_RFCLK
     entry = TAILQ_FIRST(&g_ble_ll_sched_q);
     if (entry == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     } else {
         sch = entry;
     }
-#else
-    /* Get first on list */
-    sch = TAILQ_FIRST(&g_ble_ll_sched_q);
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -1032,17 +1008,12 @@ ble_ll_sched_sync(struct ble_ll_sched_item *sch,
     }
 
 done:
-#ifdef BLE_XCVR_RFCLK
     entry = TAILQ_FIRST(&g_ble_ll_sched_q);
     if (entry == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     } else {
         sch = entry;
     }
-#else
-    /* Get first on list */
-    sch = TAILQ_FIRST(&g_ble_ll_sched_q);
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -1106,11 +1077,9 @@ ble_ll_sched_adv_new(struct ble_ll_sched_item *sch, ble_ll_sched_adv_new_cb cb,
         cb((struct ble_ll_adv_sm *)orig->cb_arg, adv_start, arg);
     }
 
-#ifdef BLE_XCVR_RFCLK
     if (orig == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     }
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -1130,9 +1099,7 @@ ble_ll_sched_periodic_adv(struct ble_ll_sched_item *sch, uint32_t *start,
     uint32_t adv_start;
     uint32_t duration;
     struct ble_ll_sched_item *entry;
-#ifdef BLE_XCVR_RFCLK
     struct ble_ll_sched_item *orig = sch;
-#endif
 
     /* Get length of schedule item */
     duration = sch->end_time - sch->start_time;
@@ -1181,11 +1148,9 @@ ble_ll_sched_periodic_adv(struct ble_ll_sched_item *sch, uint32_t *start,
         *start = adv_start;
     }
 
-#ifdef BLE_XCVR_RFCLK
     if (orig == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     }
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -1308,11 +1273,9 @@ ble_ll_sched_adv_reschedule(struct ble_ll_sched_item *sch, uint32_t *start,
         sch->end_time = sch->start_time + duration;
         *start = sch->start_time;
 
-#ifdef BLE_XCVR_RFCLK
         if (sch == TAILQ_FIRST(&g_ble_ll_sched_q)) {
-            ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+            ble_ll_rfmgmt_sched_changed(sch);
         }
-#endif
     }
 
     OS_EXIT_CRITICAL(sr);
@@ -1349,6 +1312,8 @@ ble_ll_sched_adv_resched_pdu(struct ble_ll_sched_item *sch)
         sch->enqueued = 1;
     }
 
+    ble_ll_rfmgmt_sched_changed(TAILQ_FIRST(&g_ble_ll_sched_q));
+
     OS_EXIT_CRITICAL(sr);
     os_cputime_timer_start(&g_ble_ll_sched_timer, sch->start_time);
     return 0;
@@ -1392,6 +1357,7 @@ ble_ll_sched_rmv_elem(struct ble_ll_sched_item *sch)
             if (first) {
                 os_cputime_timer_start(&g_ble_ll_sched_timer, first->start_time);
             }
+            ble_ll_rfmgmt_sched_changed(first);
         }
     }
     OS_EXIT_CRITICAL(sr);
@@ -1432,6 +1398,7 @@ ble_ll_sched_rmv_elem_type(uint8_t type, sched_remove_cb_func remove_cb)
         if (first) {
             os_cputime_timer_start(&g_ble_ll_sched_timer, first->start_time);
         }
+        ble_ll_rfmgmt_sched_changed(first);
     }
 
     OS_EXIT_CRITICAL(sr);
@@ -1548,6 +1515,7 @@ ble_ll_sched_run(void *arg)
         if (sch) {
             os_cputime_timer_start(&g_ble_ll_sched_timer, sch->start_time);
         }
+        ble_ll_rfmgmt_sched_changed(sch);
     }
 }
 
@@ -1580,64 +1548,6 @@ ble_ll_sched_next_time(uint32_t *next_event_time)
     return rc;
 }
 
-#ifdef BLE_XCVR_RFCLK
-/**
- * Checks to see if we need to restart the cputime timer which starts the
- * rf clock settling.
- *
- * NOTE: Should only be called from the Link Layer task!
- *
- * Context: Link-Layer task.
- *
- */
-void
-ble_ll_sched_rfclk_chk_restart(void)
-{
-    os_sr_t sr;
-    uint8_t ll_state;
-    int32_t time_till_next;
-    uint32_t next_time;
-
-    OS_ENTER_CRITICAL(sr);
-    ll_state = ble_ll_state_get();
-    if (ble_ll_sched_next_time(&next_time)) {
-        /*
-         * If the time until the next event is too close, no need to start
-         * the timer. Leave clock on.
-         */
-        time_till_next = (int32_t)(next_time - os_cputime_get32());
-        if (time_till_next > g_ble_ll_data.ll_xtal_ticks) {
-            /*
-             * XXX stop timer if already scheduled since it could be set for a
-             *     scheduler item which was removed prior to calling this func.
-             *     in such case we need to make sure we are set to proper item.
-             */
-            if (g_ble_ll_data.ll_rfclk_is_sched) {
-                g_ble_ll_data.ll_rfclk_is_sched = 0;
-                os_cputime_timer_stop(&g_ble_ll_data.ll_rfclk_timer);
-            }
-            /* Restart the rfclk timer based on the next scheduled time */
-            ble_ll_xcvr_rfclk_timer_start(next_time);
-
-            /* Only disable the rfclk if doing nothing */
-            if (ll_state == BLE_LL_STATE_STANDBY) {
-                ble_ll_xcvr_rfclk_disable();
-            }
-        }
-    } else {
-        /*
-         * Only stop the timer and rfclk if doing nothing currently. If
-         * in some other state, that state will handle the timer and rfclk
-         */
-        if (ll_state == BLE_LL_STATE_STANDBY) {
-            ble_ll_xcvr_rfclk_stop();
-        }
-    }
-    OS_EXIT_CRITICAL(sr);
-}
-
-#endif
-
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_LL_EXT_ADV)
 /**
  * Called to check if there is place for a planned scan req.
@@ -1784,16 +1694,12 @@ done:
     }
 
     /* Get head of list to restart timer */
-#ifdef BLE_XCVR_RFCLK
     entry = TAILQ_FIRST(&g_ble_ll_sched_q);
     if (entry == sch) {
-        ble_ll_xcvr_rfclk_timer_start(sch->start_time);
+        ble_ll_rfmgmt_sched_changed(sch);
     } else {
         sch = entry;
     }
-#else
-    sch = TAILQ_FIRST(&g_ble_ll_sched_q);
-#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -1852,9 +1758,7 @@ done:
     /* Get head of list to restart timer */
     sch = TAILQ_FIRST(&g_ble_ll_sched_q);
 
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_xcvr_rfclk_timer_start(sch->start_time);
-#endif
+    ble_ll_rfmgmt_sched_changed(sch);
 
     OS_EXIT_CRITICAL(sr);
 
diff --git a/nimble/controller/src/ble_ll_sync.c b/nimble/controller/src/ble_ll_sync.c
index f5d22fc..3861075 100644
--- a/nimble/controller/src/ble_ll_sync.c
+++ b/nimble/controller/src/ble_ll_sync.c
@@ -31,6 +31,7 @@
 #include "controller/ble_ll_whitelist.h"
 #include "controller/ble_ll_scan.h"
 #include "controller/ble_ll_resolv.h"
+#include "controller/ble_ll_rfmgmt.h"
 
 #include "nimble/ble.h"
 #include "nimble/hci_common.h"
@@ -201,9 +202,7 @@ ble_ll_sync_sm_clear(struct ble_ll_sync_sm *sm)
         ble_ll_scan_chk_resume();
     }
 
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
+    ble_ll_rfmgmt_release();
 
     BLE_LL_ASSERT(sm->sync_ev_end.ev.ev_queued == 0);
     BLE_LL_ASSERT(sm->sch.enqueued == 0);
@@ -990,15 +989,13 @@ ble_ll_sync_rx_pkt_in(struct os_mbuf *rxpdu, struct ble_mbuf_hdr *hdr)
 
     BLE_LL_ASSERT(sm);
 
+    ble_ll_rfmgmt_release();
+
     /* this could happen if sync was cancelled or terminated while pkt_in was
      * already in LL queue, just drop in that case
      */
     if (!sm->flags) {
         ble_ll_scan_chk_resume();
-
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
         return;
     }
 
@@ -1044,10 +1041,6 @@ ble_ll_sync_rx_pkt_in(struct os_mbuf *rxpdu, struct ble_mbuf_hdr *hdr)
         /* if chain was scheduled we don't end event yet */
         /* TODO should we check resume only if offset is high? */
         ble_ll_scan_chk_resume();
-
-#ifdef BLE_XCVR_RFCLK
-        ble_ll_sched_rfclk_chk_restart();
-#endif
         return;
     }
 
@@ -1152,6 +1145,8 @@ ble_ll_sync_event_end(struct ble_npl_event *ev)
     sm = ble_npl_event_get_arg(ev);
     BLE_LL_ASSERT(sm);
 
+    ble_ll_rfmgmt_release();
+
     if (sm->flags & BLE_LL_SYNC_SM_FLAG_ESTABLISHING) {
         ble_ll_sync_check_failed(sm);
     }
@@ -1159,10 +1154,6 @@ ble_ll_sync_event_end(struct ble_npl_event *ev)
     /* Check if we need to resume scanning */
     ble_ll_scan_chk_resume();
 
-#ifdef BLE_XCVR_RFCLK
-    ble_ll_sched_rfclk_chk_restart();
-#endif
-
     /* Remove any end events that might be enqueued */
     ble_npl_eventq_remove(&g_ble_ll_data.ll_evq, &sm->sync_ev_end);
 
diff --git a/nimble/controller/src/ble_ll_xcvr.c b/nimble/controller/src/ble_ll_xcvr.c
deleted file mode 100644
index d8cdad5..0000000
--- a/nimble/controller/src/ble_ll_xcvr.c
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * 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 <stddef.h>
-#include <assert.h>
-#include <stddef.h>
-#include "syscfg/syscfg.h"
-#include "os/os_cputime.h"
-#include "controller/ble_phy.h"
-#include "controller/ble_ll.h"
-#include "controller/ble_ll_xcvr.h"
-
-#ifdef BLE_XCVR_RFCLK
-int
-ble_ll_xcvr_rfclk_state(void)
-{
-    uint32_t expiry;
-
-    OS_ASSERT_CRITICAL();
-    if (g_ble_ll_data.ll_rfclk_state == BLE_RFCLK_STATE_ON) {
-        expiry = g_ble_ll_data.ll_rfclk_start_time;
-        if ((int32_t)(os_cputime_get32() - expiry) >
-                g_ble_ll_data.ll_xtal_ticks) {
-            g_ble_ll_data.ll_rfclk_state = BLE_RFCLK_STATE_SETTLED;
-        }
-    }
-    return g_ble_ll_data.ll_rfclk_state;
-}
-
-void
-ble_ll_xcvr_rfclk_enable(void)
-{
-    OS_ASSERT_CRITICAL();
-    if (g_ble_ll_data.ll_rfclk_state == BLE_RFCLK_STATE_OFF) {
-        g_ble_ll_data.ll_rfclk_state = BLE_RFCLK_STATE_ON;
-        ble_phy_rfclk_enable();
-    }
-}
-
-void
-ble_ll_xcvr_rfclk_disable(void)
-{
-    OS_ASSERT_CRITICAL();
-    if (g_ble_ll_data.ll_rfclk_state != BLE_RFCLK_STATE_OFF) {
-        ble_phy_rfclk_disable();
-        g_ble_ll_data.ll_rfclk_state = BLE_RFCLK_STATE_OFF;
-    }
-}
-
-void
-ble_ll_xcvr_rfclk_stop(void)
-{
-    OS_ASSERT_CRITICAL();
-    g_ble_ll_data.ll_rfclk_is_sched = 0;
-    os_cputime_timer_stop(&g_ble_ll_data.ll_rfclk_timer);
-    ble_ll_xcvr_rfclk_disable();
-}
-
-uint32_t
-ble_ll_xcvr_rfclk_time_till_settled(void)
-{
-    int32_t dt;
-    uint32_t rc;
-
-    rc = 0;
-    if (g_ble_ll_data.ll_rfclk_state == BLE_RFCLK_STATE_ON) {
-        dt = (int32_t)(os_cputime_get32() - g_ble_ll_data.ll_rfclk_start_time);
-        BLE_LL_ASSERT(dt >= 0);
-        if (dt < g_ble_ll_data.ll_xtal_ticks) {
-            rc = g_ble_ll_data.ll_xtal_ticks - (uint32_t)dt;
-        }
-    }
-
-    return rc;
-}
-
-/**
- * Called when the timer to turn on the RF CLOCK expires. This function checks
- * the state of the clock. If the clock is off, the clock is turned on.
- * Otherwise, we just exit.
- *
- * Context: Interrupt
- *
- * @param arg
- */
-void
-ble_ll_xcvr_rfclk_timer_exp(void *arg)
-{
-    g_ble_ll_data.ll_rfclk_is_sched = 0;
-
-    if (g_ble_ll_data.ll_rfclk_state == BLE_RFCLK_STATE_OFF) {
-        ble_ll_xcvr_rfclk_start_now();
-    }
-}
-
-/**
- * This API is used to turn on the rfclock without setting the cputime timer to
- * start the clock at some later point.
- *
- * NOTE: presumes that the state of the rf clock was checked prior to calling.
- *
- * @param now
- */
-void
-ble_ll_xcvr_rfclk_start_now(void)
-{
-    BLE_LL_ASSERT(g_ble_ll_data.ll_rfclk_state == BLE_RFCLK_STATE_OFF);
-
-    ble_ll_xcvr_rfclk_enable();
-    g_ble_ll_data.ll_rfclk_start_time = os_cputime_get32();
-}
-
-/**
- * Starts the timer that will turn the rf clock on. The 'cputime' is
- * the time at which the clock needs to be settled.
- *
- * @param cputime   Time at which rfclock should be on and settled.
- */
-void
-ble_ll_xcvr_rfclk_timer_start(uint32_t cputime)
-{
-    OS_ASSERT_CRITICAL();
-
-    /*
-     * If we are currently in an advertising event or a connection event,
-     * no need to start the cputime timer
-     */
-    if ((g_ble_ll_data.ll_state == BLE_LL_STATE_ADV) ||
-        (g_ble_ll_data.ll_state == BLE_LL_STATE_CONNECTION)) {
-        return;
-    }
-
-    /* Account for the settling time */
-    cputime -= g_ble_ll_data.ll_xtal_ticks;
-
-    /*
-     * If the timer is on the list, we need to see if its expiry is before
-     * 'cputime'. If the expiry is before, no need to do anything. If it
-     * is after, we need to stop the timer and start at new time.
-     */
-    if (g_ble_ll_data.ll_rfclk_is_sched) {
-        if ((int32_t)(cputime - g_ble_ll_data.ll_rfclk_sched_time) >= 0) {
-            return;
-        }
-        os_cputime_timer_stop(&g_ble_ll_data.ll_rfclk_timer);
-    }
-
-    g_ble_ll_data.ll_rfclk_is_sched = 1;
-    g_ble_ll_data.ll_rfclk_sched_time = cputime;
-    os_cputime_timer_start(&g_ble_ll_data.ll_rfclk_timer, cputime);
-}
-#endif
diff --git a/nimble/drivers/nrf51/src/ble_phy.c b/nimble/drivers/nrf51/src/ble_phy.c
index eef3763..30a0059 100644
--- a/nimble/drivers/nrf51/src/ble_phy.c
+++ b/nimble/drivers/nrf51/src/ble_phy.c
@@ -844,17 +844,6 @@ ble_phy_init(void)
 {
     int rc;
 
-#if !defined(BLE_XCVR_RFCLK)
-    /* BLE wants the HFXO on all the time in this case */
-    ble_phy_rfclk_enable();
-
-    /*
-     * XXX: I do not think we need to wait for settling time here since
-     * we will probably not use the radio for longer than the settling time
-     * and it will only degrade performance. Might want to wait here though.
-     */
-#endif
-
     /* Set phy channel to an invalid channel so first set channel works */
     g_ble_phy_data.phy_chan = BLE_PHY_NUM_CHANS;
 
diff --git a/nimble/drivers/nrf52/src/ble_phy.c b/nimble/drivers/nrf52/src/ble_phy.c
index 2cca6c7..725dac1 100644
--- a/nimble/drivers/nrf52/src/ble_phy.c
+++ b/nimble/drivers/nrf52/src/ble_phy.c
@@ -1375,17 +1375,6 @@ ble_phy_init(void)
 
     g_ble_phy_data.rx_pwr_compensation = 0;
 
-#if !defined(BLE_XCVR_RFCLK)
-    /* BLE wants the HFXO on all the time in this case */
-    ble_phy_rfclk_enable();
-
-    /*
-     * XXX: I do not think we need to wait for settling time here since
-     * we will probably not use the radio for longer than the settling time
-     * and it will only degrade performance. Might want to wait here though.
-     */
-#endif
-
     /* Set phy channel to an invalid channel so first set channel works */
     g_ble_phy_data.phy_chan = BLE_PHY_NUM_CHANS;