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 2022/02/22 19:53:02 UTC

[mynewt-nimble] 03/05: nimble/ll: Add new connection strict scheduling (css)

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 22f5201717bd5709d65f371aae775bbb8bc77935
Author: Andrzej Kaczmarek <an...@codecoup.pl>
AuthorDate: Thu Feb 3 23:40:45 2022 +0100

    nimble/ll: Add new connection strict scheduling (css)
    
    This adds new connection strict scheduling code. The idea is based on
    old code, but it was rewritten from scratch to allow more control over
    scheduling.
    
    Scheduling policy in css mode is defined by slot duration and number of
    slots per period (which implicitly defined period duration). Each
    connection in central role is only scheduled on slot boundary and has
    connection interval equal to period duration. This means there can be
    up to number-of-slots connection in central role established.
    
    The main difference to the previous implementation is more control over
    slot allocation for each connection. In particular, it's possible to
    assing an arbitrary slot index to each connection and thus adjust max
    connection event for each connection, depending on requirements.
---
 nimble/controller/include/controller/ble_ll_conn.h |   6 +
 .../controller/include/controller/ble_ll_sched.h   |   8 +
 nimble/controller/src/ble_ll_conn.c                | 186 ++++++++++++++++++++-
 nimble/controller/src/ble_ll_conn_hci.c            |  47 ++++++
 nimble/controller/src/ble_ll_conn_priv.h           |  13 ++
 nimble/controller/src/ble_ll_ctrl.c                |  10 ++
 nimble/controller/src/ble_ll_sched.c               | 138 ++++++++++++++-
 nimble/controller/syscfg.yml                       |  21 +++
 8 files changed, 417 insertions(+), 12 deletions(-)

diff --git a/nimble/controller/include/controller/ble_ll_conn.h b/nimble/controller/include/controller/ble_ll_conn.h
index 9075e7e..133824b 100644
--- a/nimble/controller/include/controller/ble_ll_conn.h
+++ b/nimble/controller/include/controller/ble_ll_conn.h
@@ -351,6 +351,12 @@ struct ble_ll_conn_sm
     uint16_t sync_transfer_skip;
     uint32_t sync_transfer_sync_timeout;
 #endif
+
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    uint16_t css_slot_idx;
+    uint16_t css_slot_idx_pending;
+    uint8_t css_period_idx;
+#endif
 };
 
 /* Flags */
diff --git a/nimble/controller/include/controller/ble_ll_sched.h b/nimble/controller/include/controller/ble_ll_sched.h
index b3f7d19..a79262a 100644
--- a/nimble/controller/include/controller/ble_ll_sched.h
+++ b/nimble/controller/include/controller/ble_ll_sched.h
@@ -175,6 +175,14 @@ void ble_ll_sched_stop(void);
 int ble_ll_sched_dtm(struct ble_ll_sched_item *sch);
 #endif
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+void ble_ll_sched_css_set_params(uint32_t slot_us, uint32_t period_slots);
+void ble_ll_sched_css_set_conn_anchor(struct ble_ll_conn_sm *connsm);
+uint32_t ble_ll_sched_css_get_slot_us(void);
+uint32_t ble_ll_sched_css_get_period_slots(void);
+uint32_t ble_ll_sched_css_get_conn_interval_us(void);
+#endif
+
 #ifdef __cplusplus
 }
 #endif
diff --git a/nimble/controller/src/ble_ll_conn.c b/nimble/controller/src/ble_ll_conn.c
index 0243577..6bd9758 100644
--- a/nimble/controller/src/ble_ll_conn.c
+++ b/nimble/controller/src/ble_ll_conn.c
@@ -46,6 +46,10 @@
 extern void bletest_completed_pkt(uint16_t handle);
 #endif
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+struct ble_ll_conn_sm *g_ble_ll_conn_css_ref;
+#endif
+
 /* XXX TODO
  * 1) I think if we are initiating and we already have a connection with
  * a device that we will still try and connect to it. Fix this.
@@ -388,6 +392,97 @@ ble_ll_conn_cth_flow_process_cmd(const uint8_t *cmdbuf)
 }
 #endif
 
+
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+static uint16_t g_ble_ll_conn_css_next_slot = BLE_LL_CONN_CSS_NO_SLOT;
+
+void
+ble_ll_conn_css_set_next_slot(uint16_t slot_idx)
+{
+    g_ble_ll_conn_css_next_slot = slot_idx;
+}
+
+uint16_t
+ble_ll_conn_css_get_next_slot(void)
+{
+    struct ble_ll_conn_sm *connsm;
+    uint16_t slot_idx = 0;
+
+    if (g_ble_ll_conn_css_next_slot != BLE_LL_CONN_CSS_NO_SLOT) {
+        return g_ble_ll_conn_css_next_slot;
+    }
+
+    /* CSS connections are sorted in active conn list so just need to find 1st
+     * free value.
+     */
+    SLIST_FOREACH(connsm, &g_ble_ll_conn_active_list, act_sle) {
+        if ((connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) &&
+            (connsm->css_slot_idx != slot_idx) &&
+            (connsm->css_slot_idx_pending != slot_idx)) {
+            break;
+        }
+        slot_idx++;
+    }
+
+    if (slot_idx >= ble_ll_sched_css_get_period_slots()) {
+        slot_idx = BLE_LL_CONN_CSS_NO_SLOT;
+    }
+
+    return slot_idx;
+}
+
+int
+ble_ll_conn_css_is_slot_busy(uint16_t slot_idx)
+{
+    struct ble_ll_conn_sm *connsm;
+
+    SLIST_FOREACH(connsm, &g_ble_ll_conn_active_list, act_sle) {
+        if ((connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) &&
+            ((connsm->css_slot_idx == slot_idx) ||
+             (connsm->css_slot_idx_pending == slot_idx))) {
+            return 1;
+        }
+    }
+
+    return 0;
+}
+
+int
+ble_ll_conn_css_move(struct ble_ll_conn_sm *connsm, uint16_t slot_idx)
+{
+    int16_t slot_diff;
+    uint32_t offset;
+    int rc;
+
+    /* Assume connsm and slot_idx are valid */
+    BLE_LL_ASSERT(connsm->conn_state != BLE_LL_CONN_STATE_IDLE);
+    BLE_LL_ASSERT(connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL);
+    BLE_LL_ASSERT((slot_idx < ble_ll_sched_css_get_period_slots()) ||
+                  (slot_idx != BLE_LL_CONN_CSS_NO_SLOT));
+
+    slot_diff = slot_idx - connsm->css_slot_idx;
+
+    if (slot_diff > 0) {
+        offset = slot_diff * ble_ll_sched_css_get_slot_us() /
+                  BLE_LL_CONN_ITVL_USECS;
+    } else {
+        offset = (ble_ll_sched_css_get_period_slots() + slot_diff) *
+                 ble_ll_sched_css_get_slot_us() / BLE_LL_CONN_ITVL_USECS;
+    }
+
+    if (offset >= 0xffff) {
+        return -1;
+    }
+
+    rc = ble_ll_conn_move_anchor(connsm, offset);
+    if (!rc) {
+        connsm->css_slot_idx_pending = slot_idx;
+    }
+
+    return rc;
+}
+#endif
+
 #if (BLE_LL_BT5_PHY_SUPPORTED == 1)
 /**
  * Checks to see if we should start a PHY update procedure
@@ -1720,6 +1815,10 @@ void
 ble_ll_conn_sm_new(struct ble_ll_conn_sm *connsm)
 {
     struct ble_ll_conn_global_params *conn_params;
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    struct ble_ll_conn_sm *connsm_css_prev = NULL;
+    struct ble_ll_conn_sm *connsm_css;
+#endif
 
     /* Reset following elements */
     connsm->csmflags.conn_flags = 0;
@@ -1813,7 +1912,35 @@ ble_ll_conn_sm_new(struct ble_ll_conn_sm *connsm)
 #endif
 
     /* Add to list of active connections */
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    if (connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) {
+        /* We will insert sorted by css_slot_idx to make finding free slot
+         * easier.
+         */
+        SLIST_FOREACH(connsm_css, &g_ble_ll_conn_active_list, act_sle) {
+            if ((connsm_css->conn_role == BLE_LL_CONN_ROLE_CENTRAL) &&
+                (connsm_css->css_slot_idx > connsm->css_slot_idx)) {
+                if (connsm_css_prev) {
+                    SLIST_INSERT_AFTER(connsm_css_prev, connsm, act_sle);
+                }
+                break;
+            }
+            connsm_css_prev = connsm_css;
+        }
+
+        if (!connsm_css_prev) {
+            /* List was empty or need to insert before 1st connection */
+            SLIST_INSERT_HEAD(&g_ble_ll_conn_active_list, connsm, act_sle);
+        } else if (!connsm_css) {
+            /* Insert at the end of list */
+            SLIST_INSERT_AFTER(connsm_css_prev, connsm, act_sle);
+        }
+    } else {
+        SLIST_INSERT_HEAD(&g_ble_ll_conn_active_list, connsm, act_sle);
+    }
+#else
     SLIST_INSERT_HEAD(&g_ble_ll_conn_active_list, connsm, act_sle);
+#endif
 }
 
 void
@@ -1903,6 +2030,20 @@ ble_ll_conn_end(struct ble_ll_conn_sm *connsm, uint8_t ble_err)
     /* Remove from the active connection list */
     SLIST_REMOVE(&g_ble_ll_conn_active_list, connsm, ble_ll_conn_sm, act_sle);
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    /* If current connection was reference for CSS, we need to find another
+     * one. It does not matter which one we'll pick.
+     */
+    if (connsm == g_ble_ll_conn_css_ref) {
+        SLIST_FOREACH(g_ble_ll_conn_css_ref, &g_ble_ll_conn_active_list,
+                      act_sle) {
+            if (g_ble_ll_conn_css_ref->conn_role == BLE_LL_CONN_ROLE_CENTRAL) {
+                break;
+            }
+        }
+    }
+#endif
+
 #if MYNEWT_VAL(BLE_LL_CFG_FEAT_CTRL_TO_HOST_FLOW_CONTROL)
     ble_ll_conn_cth_flow_free_credit(connsm, connsm->cth_flow_pending);
 #endif
@@ -2039,6 +2180,7 @@ ble_ll_conn_next_event(struct ble_ll_conn_sm *connsm)
     uint32_t max_ww;
 #endif
     struct ble_ll_conn_upd_req *upd;
+    uint8_t skip_anchor_calc = 0;
     uint32_t usecs;
 
     /* XXX: deal with connection request procedure here as well */
@@ -2083,15 +2225,32 @@ ble_ll_conn_next_event(struct ble_ll_conn_sm *connsm)
     }
     connsm->event_cntr += latency;
 
-    /* Set next connection event start time */
-    /* We can use pre-calculated values for one interval if latency is 1. */
-    if (latency == 1) {
-        connsm->anchor_point += connsm->conn_itvl_ticks;
-        ble_ll_tmr_add_u(&connsm->anchor_point, &connsm->anchor_point_usecs,
-                         connsm->conn_itvl_usecs);
-    } else {
-        ble_ll_tmr_add(&connsm->anchor_point, &connsm->anchor_point_usecs,
-                       itvl);
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    if (connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) {
+        connsm->css_period_idx += latency;
+
+        /* If this is non-reference connection, we set anchor from reference
+         * instead of calculating manually.
+         */
+        if (g_ble_ll_conn_css_ref != connsm) {
+            ble_ll_sched_css_set_conn_anchor(connsm);
+            skip_anchor_calc = 1;
+        }
+    }
+#endif
+
+    if (!skip_anchor_calc) {
+        /* Calculate next anchor point for connection.
+         * We can use pre-calculated values for one interval if latency is 1.
+         */
+        if (latency == 1) {
+            connsm->anchor_point += connsm->conn_itvl_ticks;
+            ble_ll_tmr_add_u(&connsm->anchor_point, &connsm->anchor_point_usecs,
+                             connsm->conn_itvl_usecs);
+        } else {
+            ble_ll_tmr_add(&connsm->anchor_point, &connsm->anchor_point_usecs,
+                           itvl);
+        }
     }
 
     /*
@@ -2134,6 +2293,15 @@ ble_ll_conn_next_event(struct ble_ll_conn_sm *connsm)
         /* Reset the starting point of the connection supervision timeout */
         connsm->last_rxd_pdu_cputime = connsm->anchor_point;
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+        if (connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) {
+            BLE_LL_ASSERT(connsm->css_slot_idx_pending !=
+                          BLE_LL_CONN_CSS_NO_SLOT);
+            connsm->css_slot_idx = connsm->css_slot_idx_pending;
+            connsm->css_slot_idx_pending = BLE_LL_CONN_CSS_NO_SLOT;
+        }
+#endif
+
         /* Reset update scheduled flag */
         connsm->csmflags.cfbit.conn_update_sched = 0;
     }
diff --git a/nimble/controller/src/ble_ll_conn_hci.c b/nimble/controller/src/ble_ll_conn_hci.c
index cd3deae..ac21255 100644
--- a/nimble/controller/src/ble_ll_conn_hci.c
+++ b/nimble/controller/src/ble_ll_conn_hci.c
@@ -513,6 +513,9 @@ ble_ll_conn_hci_create(const uint8_t *cmdbuf, uint8_t len)
     struct ble_ll_conn_sm *connsm;
     uint16_t conn_itvl_min;
     uint16_t conn_itvl_max;
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    uint16_t css_slot_idx;
+#endif
     int rc;
 
     if (len < sizeof(*cmd)) {
@@ -529,6 +532,14 @@ ble_ll_conn_hci_create(const uint8_t *cmdbuf, uint8_t len)
         return BLE_ERR_CMD_DISALLOWED;
     }
 
+
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    css_slot_idx = ble_ll_conn_css_get_next_slot();
+    if (css_slot_idx == BLE_LL_CONN_CSS_NO_SLOT) {
+        return BLE_ERR_MEM_CAPACITY;
+    }
+#endif
+
     cc_scan.own_addr_type = cmd->own_addr_type;
     cc_scan.filter_policy = cmd->filter_policy;
     if (cc_scan.filter_policy == 0) {
@@ -560,7 +571,15 @@ ble_ll_conn_hci_create(const uint8_t *cmdbuf, uint8_t len)
         return BLE_ERR_INV_HCI_CMD_PARMS;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    cc_params.conn_itvl = ble_ll_sched_css_get_conn_interval_us();
+    if ((cc_params.conn_itvl < conn_itvl_min) ||
+        (cc_params.conn_itvl > conn_itvl_max)) {
+        return BLE_ERR_INV_HCI_CMD_PARMS;
+    }
+#else
     cc_params.conn_itvl = conn_itvl_max;
+#endif
     cc_params.conn_latency = le16toh(cmd->conn_latency);
     cc_params.supervision_timeout = le16toh(cmd->tmo);
     cc_params.min_ce_len = le16toh(cmd->min_ce);
@@ -582,6 +601,11 @@ ble_ll_conn_hci_create(const uint8_t *cmdbuf, uint8_t len)
         return BLE_ERR_CONN_LIMIT;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    connsm->css_slot_idx = css_slot_idx;
+    connsm->css_slot_idx_pending = BLE_LL_CONN_CSS_NO_SLOT;
+#endif
+
     /* Initialize state machine in central role and start state machine */
     ble_ll_conn_central_init(connsm, &cc_scan, &cc_params);
     ble_ll_conn_sm_new(connsm);
@@ -640,7 +664,15 @@ ble_ll_conn_hci_ext_create_parse_params(const struct conn_params *params,
         return BLE_ERR_INV_HCI_CMD_PARMS;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    cc_params->conn_itvl = ble_ll_sched_css_get_conn_interval_us();
+    if ((cc_params->conn_itvl < conn_itvl_min) ||
+        (cc_params->conn_itvl > conn_itvl_max)) {
+        return BLE_ERR_INV_HCI_CMD_PARMS;
+    }
+#else
     cc_params->conn_itvl = conn_itvl_max;
+#endif
     cc_params->conn_latency = le16toh(params->conn_latency);
     cc_params->supervision_timeout = le16toh(params->supervision_timeout);
     cc_params->min_ce_len = le16toh(params->min_ce);
@@ -699,6 +731,9 @@ ble_ll_conn_hci_ext_create(const uint8_t *cmdbuf, uint8_t len)
     struct ble_ll_conn_create_params *cc_params_fb;
     struct ble_ll_conn_sm *connsm;
     const struct init_phy *init_phy;
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    uint16_t css_slot_idx;
+#endif
     int rc;
 
     /* validate length */
@@ -717,6 +752,13 @@ ble_ll_conn_hci_ext_create(const uint8_t *cmdbuf, uint8_t len)
         return BLE_ERR_CMD_DISALLOWED;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    css_slot_idx = ble_ll_conn_css_get_next_slot();
+    if (css_slot_idx == BLE_LL_CONN_CSS_NO_SLOT) {
+        return BLE_ERR_MEM_CAPACITY;
+    }
+#endif
+
     cc_scan.own_addr_type = cmd->own_addr_type;
     cc_scan.filter_policy = cmd->filter_policy;
     if (cc_scan.filter_policy == 0) {
@@ -770,6 +812,11 @@ ble_ll_conn_hci_ext_create(const uint8_t *cmdbuf, uint8_t len)
         return BLE_ERR_CONN_LIMIT;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    connsm->css_slot_idx = css_slot_idx;
+    connsm->css_slot_idx_pending = BLE_LL_CONN_CSS_NO_SLOT;
+#endif
+
     /* Initialize state machine in central role and start state machine */
     ble_ll_conn_central_init(connsm, &cc_scan,
                              &g_ble_ll_conn_create_sm.params[0]);
diff --git a/nimble/controller/src/ble_ll_conn_priv.h b/nimble/controller/src/ble_ll_conn_priv.h
index c51aef9..3d49f49 100644
--- a/nimble/controller/src/ble_ll_conn_priv.h
+++ b/nimble/controller/src/ble_ll_conn_priv.h
@@ -59,6 +59,10 @@ extern "C" {
 #define BLE_LL_CONN_DEF_AUTH_PYLD_TMO       (3000)
 #define BLE_LL_CONN_AUTH_PYLD_OS_TMO(x)     ble_npl_time_ms_to_ticks32((x) * 10)
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+#define BLE_LL_CONN_CSS_NO_SLOT             (UINT16_MAX)
+#endif
+
 /* Global Link Layer connection parameters */
 struct ble_ll_conn_global_params
 {
@@ -127,6 +131,7 @@ struct ble_ll_conn_create_sm {
 };
 
 extern struct ble_ll_conn_create_sm g_ble_ll_conn_create_sm;
+extern struct ble_ll_conn_sm *g_ble_ll_conn_css_ref;
 
 /* Generic interface */
 struct ble_ll_len_req;
@@ -250,6 +255,14 @@ int ble_ll_set_sync_transfer_params(const uint8_t *cmdbuf, uint8_t len,
 int ble_ll_set_default_sync_transfer_params(const uint8_t *cmdbuf, uint8_t len);
 #endif
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+void ble_ll_conn_css_set_next_slot(uint16_t slot_idx);
+uint16_t ble_ll_conn_css_get_next_slot(void);
+int ble_ll_conn_css_is_slot_busy(uint16_t slot_idx);
+int ble_ll_conn_css_move(struct ble_ll_conn_sm *connsm, uint16_t slot_idx);
+
+#endif
+
 #ifdef __cplusplus
 }
 #endif
diff --git a/nimble/controller/src/ble_ll_ctrl.c b/nimble/controller/src/ble_ll_ctrl.c
index a153a57..5212cb5 100644
--- a/nimble/controller/src/ble_ll_ctrl.c
+++ b/nimble/controller/src/ble_ll_ctrl.c
@@ -2106,6 +2106,16 @@ ble_ll_ctrl_rx_conn_param_req(struct ble_ll_conn_sm *connsm, uint8_t *dptr,
         return BLE_ERR_MAX;
     }
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    /* Reject any attempts to change connection parameters by peripheral */
+    if (connsm->conn_role == BLE_LL_CONN_ROLE_CENTRAL) {
+        rsp_opcode = BLE_LL_CTRL_REJECT_IND_EXT;
+        rspbuf[1] = BLE_LL_CTRL_CONN_PARM_REQ;
+        rspbuf[2] = BLE_ERR_UNSUPPORTED;
+        return rsp_opcode;
+    }
+#endif
+
     /* XXX: remember to deal with this on the central: if the peripheral has
      * initiated a procedure we may have received its connection parameter
      * update request and have signaled the host with an event. If that
diff --git a/nimble/controller/src/ble_ll_sched.c b/nimble/controller/src/ble_ll_sched.c
index 14c8753..eb9ed8b 100644
--- a/nimble/controller/src/ble_ll_sched.c
+++ b/nimble/controller/src/ble_ll_sched.c
@@ -45,6 +45,22 @@ int32_t g_ble_ll_sched_max_late;
 int32_t g_ble_ll_sched_max_early;
 #endif
 
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+struct ble_ll_sched_css {
+    uint32_t slot_us;
+    uint32_t period_slots;
+    uint32_t period_anchor_ticks;
+    uint8_t period_anchor_rem_us;
+    uint8_t period_anchor_idx;
+    uint16_t period_anchor_slot_idx;
+};
+
+static struct ble_ll_sched_css g_ble_ll_sched_css = {
+    .slot_us = MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED_SLOT_US),
+    .period_slots = MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED_PERIOD_SLOTS),
+};
+#endif
+
 typedef int (* ble_ll_sched_preempt_cb_t)(struct ble_ll_sched_item *sch,
                                           struct ble_ll_sched_item *item);
 
@@ -304,6 +320,9 @@ ble_ll_sched_overlaps_current(struct ble_ll_sched_item *sch)
 int
 ble_ll_sched_conn_reschedule(struct ble_ll_conn_sm *connsm)
 {
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    struct ble_ll_sched_css *css = &g_ble_ll_sched_css;
+#endif
     struct ble_ll_sched_item *sch;
 #if MYNEWT_VAL(BLE_LL_ROLE_PERIPHERAL)
     uint32_t usecs;
@@ -349,6 +368,17 @@ ble_ll_sched_conn_reschedule(struct ble_ll_conn_sm *connsm)
     }
 
     rc = ble_ll_sched_insert(sch, 0, preempt_any_except_conn);
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    /* Store new anchor point for strict scheduling if successfully scheduled
+     * reference connection.
+     */
+    if ((rc == 0) && (connsm == g_ble_ll_conn_css_ref)) {
+        css->period_anchor_idx = connsm->css_period_idx;
+        css->period_anchor_slot_idx = connsm->css_slot_idx;
+        css->period_anchor_ticks = connsm->anchor_point;
+        css->period_anchor_rem_us = connsm->anchor_point_usecs;
+    }
+#endif
 
     OS_EXIT_CRITICAL(sr);
 
@@ -372,10 +402,16 @@ int
 ble_ll_sched_conn_central_new(struct ble_ll_conn_sm *connsm,
                               struct ble_mbuf_hdr *ble_hdr, uint8_t pyld_len)
 {
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    struct ble_ll_sched_css *css = &g_ble_ll_sched_css;
+    struct ble_ll_conn_sm *connsm_ref;
+#endif
     struct ble_ll_sched_item *sch;
     uint32_t orig_start_time;
     uint32_t earliest_start;
+#if !MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
     uint32_t min_win_offset;
+#endif
     uint32_t max_delay;
     uint32_t adv_rxend;
     os_sr_t sr;
@@ -457,20 +493,60 @@ ble_ll_sched_conn_central_new(struct ble_ll_conn_sm *connsm,
         }
     }
 
+    orig_start_time = earliest_start - g_ble_ll_sched_offset_ticks;
+
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+    uint8_t rem_us;
+
+    OS_ENTER_CRITICAL(sr);
+
+    connsm_ref = g_ble_ll_conn_css_ref;
+    if (!connsm_ref) {
+        g_ble_ll_conn_css_ref = connsm;
+
+        css->period_anchor_slot_idx = connsm->css_slot_idx;
+        css->period_anchor_idx = 0;
+        css->period_anchor_ticks = adv_rxend;
+        css->period_anchor_rem_us = 0;
+
+        connsm->css_period_idx = 1;
+    } else {
+        connsm->css_period_idx = css->period_anchor_idx + 1;
+    }
+
+    ble_ll_sched_css_set_conn_anchor(connsm);
+
+    sch->start_time = connsm->anchor_point - g_ble_ll_sched_offset_ticks;
+    sch->remainder = connsm->anchor_point_usecs;
+
+    OS_EXIT_CRITICAL(sr);
+
+    sch->end_time = sch->start_time;
+    rem_us = sch->remainder;
+    ble_ll_tmr_add(&sch->end_time, &rem_us, ble_ll_sched_css_get_period_slots());
+    if (rem_us == 0) {
+        sch->end_time--;
+    }
+
+    max_delay = 0;
+
+#else
+
     sch->start_time = earliest_start - g_ble_ll_sched_offset_ticks;
     sch->end_time = earliest_start +
                     ble_ll_tmr_u2t(MYNEWT_VAL(BLE_LL_CONN_INIT_SLOTS) *
                                    BLE_LL_SCHED_USECS_PER_SLOT);
 
-    orig_start_time = sch->start_time;
-
     min_win_offset = ble_ll_tmr_u2t(MYNEWT_VAL(BLE_LL_CONN_INIT_MIN_WIN_OFFSET) *
                                     BLE_LL_SCHED_USECS_PER_SLOT);
     sch->start_time += min_win_offset;
     sch->end_time += min_win_offset;
+    sch->remainder = 0;
 
     max_delay = connsm->conn_itvl_ticks - min_win_offset;
 
+#endif
+
     OS_ENTER_CRITICAL(sr);
 
     rc = ble_ll_sched_insert(sch, max_delay, preempt_none);
@@ -480,7 +556,7 @@ ble_ll_sched_conn_central_new(struct ble_ll_conn_sm *connsm,
                              BLE_LL_CONN_TX_OFF_USECS;
 
         connsm->anchor_point = sch->start_time + g_ble_ll_sched_offset_ticks;
-        connsm->anchor_point_usecs = 0;
+        connsm->anchor_point_usecs = sch->remainder;
         connsm->ce_end_time = sch->end_time;
 
     }
@@ -1104,3 +1180,59 @@ ble_ll_sched_init(void)
 
     return 0;
 }
+
+#if MYNEWT_VAL(BLE_LL_CONN_STRICT_SCHED)
+void
+ble_ll_sched_css_set_params(uint32_t slot_us, uint32_t period_slots)
+{
+    g_ble_ll_sched_css.slot_us = slot_us;
+    g_ble_ll_sched_css.period_slots = period_slots;
+}
+
+void
+ble_ll_sched_css_set_conn_anchor(struct ble_ll_conn_sm *connsm)
+{
+    struct ble_ll_sched_css *css = &g_ble_ll_sched_css;
+    int8_t period_diff;
+    int16_t slot_diff;
+    int32_t diff;
+
+    period_diff = connsm->css_period_idx - css->period_anchor_idx;
+    slot_diff = connsm->css_slot_idx - css->period_anchor_slot_idx;
+
+    diff = (period_diff * ble_ll_sched_css_get_period_slots() + slot_diff) *
+           ble_ll_sched_css_get_slot_us();
+
+    connsm->anchor_point = css->period_anchor_ticks;
+    connsm->anchor_point_usecs = css->period_anchor_rem_us;
+
+    if (diff < 0) {
+        ble_ll_tmr_sub(&connsm->anchor_point, &connsm->anchor_point_usecs,
+                       -diff);
+    } else if (diff > 0) {
+        ble_ll_tmr_add(&connsm->anchor_point, &connsm->anchor_point_usecs,
+                       diff);
+    }
+}
+
+inline uint32_t
+ble_ll_sched_css_get_slot_us(void)
+{
+    return g_ble_ll_sched_css.slot_us;
+}
+
+inline uint32_t
+ble_ll_sched_css_get_period_slots(void)
+{
+    return g_ble_ll_sched_css.period_slots;
+}
+
+inline uint32_t
+ble_ll_sched_css_get_conn_interval_us(void)
+{
+    return ble_ll_sched_css_get_period_slots() *
+           ble_ll_sched_css_get_slot_us() /
+           BLE_LL_CONN_ITVL_USECS;
+}
+
+#endif
diff --git a/nimble/controller/syscfg.yml b/nimble/controller/syscfg.yml
index 8ecdd2c..de6b5ea 100644
--- a/nimble/controller/syscfg.yml
+++ b/nimble/controller/syscfg.yml
@@ -146,6 +146,27 @@ syscfg.defs:
             ensure interoperability with such devices set this value to 2 (or more).
         value: '0'
 
+    BLE_LL_CONN_STRICT_SCHED:
+        description: >
+            Enable connection strict scheduling (css).
+            In css mode, connections in central role are scheduled in fixed time
+            intervals called periods. Each period is divided into an arbitrary
+            number of slots and each connection anchor point is always scheduled
+            at slot boundary. This means (assuming only central connections are
+            active) it's possible to reliably schedule up to number-of-slots
+            connections each at period-duration interval, each connection will
+            be allocated at least one slot in each connection event.
+        value: 0
+    BLE_LL_CONN_STRICT_SCHED_SLOT_US:
+        description: >
+            Slot duration in microseconds. Shall be multiply of 1250us.
+        value: 3750
+    BLE_LL_CONN_STRICT_SCHED_PERIOD_SLOTS:
+        description: >
+            Number of slots per period. Duration of slot determines connection
+            interval used for each connection in central role.
+        value: 8
+
     # The number of random bytes to store
     BLE_LL_RNG_BUFSIZE:
         description: >