aboutsummaryrefslogtreecommitdiffstats
path: root/src/tbf_dl.cpp
diff options
context:
space:
mode:
Diffstat (limited to 'src/tbf_dl.cpp')
-rw-r--r--src/tbf_dl.cpp517
1 files changed, 241 insertions, 276 deletions
diff --git a/src/tbf_dl.cpp b/src/tbf_dl.cpp
index 05d5ad36..dc35b844 100644
--- a/src/tbf_dl.cpp
+++ b/src/tbf_dl.cpp
@@ -13,10 +13,6 @@
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
- *
- * You should have received a copy of the GNU General Public License
- * along with this program; if not, write to the Free Software
- * Foundation, Inc., 59 Temple Place - Suite 330, Boston, MA 02111-1307, USA.
*/
#include <bts.h>
@@ -31,9 +27,9 @@
#include <decoding.h>
#include <encoding.h>
#include <gprs_ms.h>
-#include <gprs_ms_storage.h>
#include <llc.h>
#include "pcu_utils.h"
+#include "alloc_algo.h"
extern "C" {
#include <osmocom/core/msgb.h>
@@ -94,26 +90,14 @@ static const struct rate_ctr_group_desc tbf_dl_egprs_ctrg_desc = {
tbf_dl_egprs_ctr_description,
};
-static void llc_timer_cb(void *_tbf)
-{
- struct gprs_rlcmac_dl_tbf *tbf = (struct gprs_rlcmac_dl_tbf *)_tbf;
-
- if (tbf->state_is_not(TBF_ST_FLOW))
- return;
-
- LOGPTBFDL(tbf, LOGL_DEBUG, "LLC receive timeout, requesting DL ACK\n");
-
- tbf->request_dl_ack();
-}
-
gprs_rlcmac_dl_tbf::BandWidth::BandWidth() :
dl_bw_octets(0),
dl_throughput(0),
dl_loss_lost(0),
dl_loss_received(0)
{
- timespecclear(&dl_bw_tv);
- timespecclear(&dl_loss_tv);
+ osmo_clock_gettime(CLOCK_MONOTONIC, &dl_bw_tv);
+ osmo_clock_gettime(CLOCK_MONOTONIC, &dl_loss_tv);
}
static int dl_tbf_dtor(struct gprs_rlcmac_dl_tbf *tbf)
@@ -122,10 +106,9 @@ static int dl_tbf_dtor(struct gprs_rlcmac_dl_tbf *tbf)
return 0;
}
-struct gprs_rlcmac_dl_tbf *tbf_alloc_dl_tbf(struct gprs_rlcmac_bts *bts, GprsMs *ms, int8_t use_trx, bool single_slot)
+struct gprs_rlcmac_dl_tbf *dl_tbf_alloc(struct gprs_rlcmac_bts *bts, struct GprsMs *ms)
{
struct gprs_rlcmac_dl_tbf *tbf;
- int rc;
OSMO_ASSERT(ms != NULL);
@@ -140,15 +123,7 @@ struct gprs_rlcmac_dl_tbf *tbf_alloc_dl_tbf(struct gprs_rlcmac_bts *bts, GprsMs
talloc_set_destructor(tbf, dl_tbf_dtor);
new (tbf) gprs_rlcmac_dl_tbf(bts, ms);
- rc = tbf->setup(use_trx, single_slot);
- /* if no resource */
- if (rc < 0) {
- talloc_free(tbf);
- return NULL;
- }
-
if (tbf->is_egprs_enabled()) {
- tbf->set_window_size();
tbf->m_dl_egprs_ctrs = rate_ctr_group_alloc(tbf,
&tbf_dl_egprs_ctrg_desc,
tbf->m_ctrs->idx);
@@ -167,19 +142,13 @@ struct gprs_rlcmac_dl_tbf *tbf_alloc_dl_tbf(struct gprs_rlcmac_bts *bts, GprsMs
return NULL;
}
}
-
- llist_add(tbf_trx_list((struct gprs_rlcmac_tbf *)tbf), &tbf->trx->dl_tbfs);
bts_do_rate_ctr_inc(tbf->bts, CTR_TBF_DL_ALLOCATED);
- osmo_clock_gettime(CLOCK_MONOTONIC, &tbf->m_bw.dl_bw_tv);
- osmo_clock_gettime(CLOCK_MONOTONIC, &tbf->m_bw.dl_loss_tv);
-
return tbf;
}
gprs_rlcmac_dl_tbf::~gprs_rlcmac_dl_tbf()
{
- osmo_timer_del(&m_llc_timer);
if (is_egprs_enabled()) {
rate_ctr_group_free(m_dl_egprs_ctrs);
} else {
@@ -193,91 +162,23 @@ gprs_rlcmac_dl_tbf::gprs_rlcmac_dl_tbf(struct gprs_rlcmac_bts *bts_, GprsMs *ms)
m_tx_counter(0),
m_dl_ack_requested(false),
m_last_dl_poll_fn(-1),
+ m_last_dl_poll_ack_lost(false),
m_last_dl_drained_fn(-1),
+ m_first_dl_ack_rcvd(false),
m_dl_gprs_ctrs(NULL),
m_dl_egprs_ctrs(NULL)
{
- memset(&m_llc_timer, 0, sizeof(m_llc_timer));
- osmo_timer_setup(&m_llc_timer, llc_timer_cb, this);
-}
-
-void gprs_rlcmac_dl_tbf::start_llc_timer()
-{
- if (the_pcu->vty.llc_idle_ack_csec > 0) {
- struct timespec tv;
- csecs_to_timespec(the_pcu->vty.llc_idle_ack_csec, &tv);
- osmo_timer_schedule(&m_llc_timer, tv.tv_sec, tv.tv_nsec / 1000);
- }
-}
-
-int gprs_rlcmac_dl_tbf::append_data(uint16_t pdu_delay_csec,
- const uint8_t *data, uint16_t len)
-{
- struct timespec expire_time;
-
- LOGPTBFDL(this, LOGL_DEBUG, "appending %u bytes\n", len);
-
- struct msgb *llc_msg = msgb_alloc(len, "llc_pdu_queue");
- if (!llc_msg)
- return -ENOMEM;
-
- gprs_llc_queue::calc_pdu_lifetime(bts, pdu_delay_csec, &expire_time);
- memcpy(msgb_put(llc_msg, len), data, len);
- llc_queue()->enqueue(llc_msg, &expire_time);
- start_llc_timer();
-
- if (state_is(TBF_ST_WAIT_RELEASE)) {
- LOGPTBFDL(this, LOGL_DEBUG, "in WAIT RELEASE state (T3193), so reuse TBF\n");
- establish_dl_tbf_on_pacch();
- }
-
- return 0;
-}
-
-static int tbf_new_dl_assignment(struct gprs_rlcmac_bts *bts, GprsMs *ms,
- struct gprs_rlcmac_dl_tbf **tbf)
-{
- bool ss;
- int8_t use_trx;
- struct gprs_rlcmac_ul_tbf *ul_tbf = NULL, *old_ul_tbf;
- struct gprs_rlcmac_dl_tbf *dl_tbf = NULL;
-
- ul_tbf = ms_ul_tbf(ms);
-
- /* 3GPP TS 44.060 sec 7.1.3.1 Initiation of the Packet resource request procedure:
- * "Furthermore, the mobile station shall not respond to PACKET DOWNLINK ASSIGNMENT
- * or MULTIPLE TBF DOWNLINK ASSIGNMENT messages before contention resolution is
- * completed on the mobile station side." */
- if (ul_tbf && ul_tbf->m_contention_resolution_done
- && !tbf_ul_ack_waiting_cnf_final_ack(ul_tbf)) {
- use_trx = ul_tbf->trx->trx_no;
- ss = false;
- old_ul_tbf = ul_tbf;
- } else {
- use_trx = -1;
- ss = true; /* PCH assignment only allows one timeslot */
- old_ul_tbf = NULL;
- }
+ memset(&state_fsm, 0, sizeof(state_fsm));
+ state_fsm.dl_tbf = this;
+ state_fi = osmo_fsm_inst_alloc(&tbf_dl_fsm, this, &state_fsm, LOGL_INFO, NULL);
+ OSMO_ASSERT(state_fi);
- // Create new TBF (any TRX)
-/* FIXME: Copy and paste with alloc_ul_tbf */
- /* set number of downlink slots according to multislot class */
- dl_tbf = tbf_alloc_dl_tbf(bts, ms, use_trx, ss);
+ INIT_LLIST_HEAD(&this->tx_llc_until_first_dl_ack_rcvd);
- if (!dl_tbf) {
- LOGPMS(ms, DTBF, LOGL_NOTICE, "No PDCH resource\n");
- return -EBUSY;
- }
-
- LOGPTBFDL(dl_tbf, LOGL_DEBUG, "[DOWNLINK] START\n");
-
- /* trigger downlink assignment and set state to ASSIGN.
- * we don't use old_downlink, so the possible uplink is used
- * to trigger downlink assignment. if there is no uplink,
- * AGCH is used. */
- dl_tbf->trigger_ass(old_ul_tbf);
- *tbf = dl_tbf;
- return 0;
+ /* This has to be called in child constructor because enable_egprs()
+ * uses the window() virtual function which is dependent on subclass. */
+ if (ms_mode(m_ms) != GPRS)
+ enable_egprs();
}
/**
@@ -289,41 +190,35 @@ int dl_tbf_handle(struct gprs_rlcmac_bts *bts,
const uint16_t delay_csec,
const uint8_t *data, const uint16_t len)
{
- struct gprs_rlcmac_dl_tbf *dl_tbf = NULL;
int rc;
GprsMs *ms, *ms_old;
+ bool ms_allocated = false;
/* check for existing TBF */
- ms = bts_ms_store(bts)->get_ms(tlli, tlli_old, imsi);
-
- if (ms && strlen(ms_imsi(ms)) == 0) {
- ms_old = bts_ms_store(bts)->get_ms(0, 0, imsi);
- if (ms_old && ms_old != ms) {
- /* The TLLI has changed (RAU), so there are two MS
- * objects for the same MS */
- LOGP(DTBF, LOGL_NOTICE,
- "There is a new MS object for the same MS: (0x%08x, '%s') -> (0x%08x, '%s')\n",
- ms_tlli(ms_old), ms_imsi(ms_old), ms_tlli(ms), ms_imsi(ms));
-
- ms_ref(ms_old);
-
- if (!ms_dl_tbf(ms) && ms_dl_tbf(ms_old)) {
+ ms = bts_get_ms(bts, tlli, tlli_old, imsi);
+
+ /* If we got MS by TLLI above let's see if we already have another MS
+ * object identified by IMSI and merge them */
+ if (ms) {
+ if (!ms_imsi_is_valid(ms) && imsi) {
+ ms_old = bts_get_ms_by_imsi(bts, imsi);
+ if (ms_old && ms_old != ms) {
+ /* The TLLI has changed (RAU), so there are two MS
+ * objects for the same MS */
LOGP(DTBF, LOGL_NOTICE,
- "IMSI %s, old TBF %s: moving DL TBF to new MS object\n",
- imsi, ms_dl_tbf(ms_old)->name());
- dl_tbf = ms_dl_tbf(ms_old);
- /* Move the DL TBF to the new MS */
- dl_tbf->set_ms(ms);
+ "There is a new MS object for the same MS: (0x%08x, '%s') -> (0x%08x, '%s')\n",
+ ms_tlli(ms_old), ms_imsi(ms_old), ms_tlli(ms), ms_imsi(ms));
+ ms_merge_and_clear_ms(ms, ms_old);
+ /* old_ms may no longer be available here */
}
- ms_merge_and_clear_ms(ms, ms_old);
-
- ms_unref(ms_old);
}
+ } else {
+ ms = ms_alloc(bts, __func__);
+ /* Remember we have to unref the alloc reference at the end: */
+ ms_allocated = true;
}
-
- if (!ms)
- ms = bts_alloc_ms(bts, ms_class, egprs_ms_class);
- ms_set_imsi(ms, imsi);
+ if (imsi)
+ ms_set_imsi(ms, imsi);
ms_confirm_tlli(ms, tlli);
if (!ms_ms_class(ms) && ms_class) {
ms_set_ms_class(ms, ms_class);
@@ -332,95 +227,15 @@ int dl_tbf_handle(struct gprs_rlcmac_bts *bts,
ms_set_egprs_ms_class(ms, egprs_ms_class);
}
- dl_tbf = ms_dl_tbf(ms);
- if (!dl_tbf) {
- rc = tbf_new_dl_assignment(bts, ms, &dl_tbf);
- if (rc < 0)
- return rc;
+ rc = ms_append_llc_dl_data(ms, delay_csec, data, len);
+ if (ms_allocated) {
+ ms_unref(ms, __func__);
+ /* Here "ms" may be freed if ms_append_llc_dl_data() failed to
+ * allocate a DL TBF and it has no more TBFs attached */
}
-
- rc = dl_tbf->append_data(delay_csec, data, len);
-
return rc;
}
-struct msgb *gprs_rlcmac_dl_tbf::llc_dequeue(bssgp_bvc_ctx *bctx)
-{
- struct msgb *msg;
- struct timespec tv_now, tv_now2;
- uint32_t octets = 0, frames = 0;
- struct timespec hyst_delta = {0, 0};
- const unsigned keep_small_thresh = 60;
- const MetaInfo *info;
-
- if (the_pcu->vty.llc_discard_csec)
- csecs_to_timespec(the_pcu->vty.llc_discard_csec, &hyst_delta);
-
- osmo_clock_gettime(CLOCK_MONOTONIC, &tv_now);
- timespecadd(&tv_now, &hyst_delta, &tv_now2);
-
- while ((msg = llc_queue()->dequeue(&info))) {
- const struct timespec *tv_disc = &info->expire_time;
- const struct timespec *tv_recv = &info->recv_time;
-
- gprs_bssgp_update_queue_delay(tv_recv, &tv_now);
-
- if (ms() && ms_codel_state(ms())) {
- int bytes = llc_queue_octets(llc_queue());
- if (gprs_codel_control(ms_codel_state(ms()),
- tv_recv, &tv_now, bytes))
- goto drop_frame;
- }
-
- /* Is the age below the low water mark? */
- if (!gprs_llc_queue::is_frame_expired(&tv_now2, tv_disc))
- break;
-
- /* Is the age below the high water mark */
- if (!gprs_llc_queue::is_frame_expired(&tv_now, tv_disc)) {
- /* Has the previous message not been dropped? */
- if (frames == 0)
- break;
-
- /* Hysteresis mode, try to discard LLC messages until
- * the low water mark has been reached */
-
- /* Check whether to abort the hysteresis mode */
-
- /* Is the frame small, perhaps only a TCP ACK? */
- if (msg->len <= keep_small_thresh)
- break;
-
- /* Is it a GMM message? */
- if (!gprs_llc::is_user_data_frame(msg->data, msg->len))
- break;
- }
-
- bts_do_rate_ctr_inc(bts, CTR_LLC_FRAME_TIMEDOUT);
-drop_frame:
- frames++;
- octets += msg->len;
- msgb_free(msg);
- bts_do_rate_ctr_inc(bts, CTR_LLC_FRAME_DROPPED);
- continue;
- }
-
- if (frames) {
- LOGPTBFDL(this, LOGL_NOTICE, "Discarding LLC PDU "
- "because lifetime limit reached, "
- "count=%u new_queue_size=%zu\n",
- frames, llc_queue_size(llc_queue()));
- if (frames > 0xff)
- frames = 0xff;
- if (octets > 0xffffff)
- octets = 0xffffff;
- if (bctx)
- bssgp_tx_llc_discarded(bctx, tlli(), frames, octets);
- }
-
- return msg;
-}
-
bool gprs_rlcmac_dl_tbf::restart_bsn_cycle()
{
/* If V(S) == V(A) and finished state, we would have received
@@ -578,7 +393,7 @@ int gprs_rlcmac_dl_tbf::take_next_bsn(uint32_t fn,
* Create DL data block
* The messages are fragmented and forwarded as data blocks.
*/
-struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(uint32_t fn, uint8_t ts, enum mcs_kind req_mcs_kind)
+struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(uint32_t fn, const struct gprs_rlcmac_pdch *pdch, enum mcs_kind req_mcs_kind)
{
int bsn, bsn2 = -1;
bool may_combine;
@@ -593,35 +408,132 @@ struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(uint32_t fn, uint8_t ts,
if (may_combine)
bsn2 = take_next_bsn(fn, bsn, req_mcs_kind, &may_combine);
- return create_dl_acked_block(fn, ts, bsn, bsn2);
+ return create_dl_acked_block(fn, pdch, bsn, bsn2);
+}
+
+void gprs_rlcmac_dl_tbf::apply_allocated_resources(const struct alloc_resources_res *res)
+{
+ uint8_t ts;
+
+ if (this->trx)
+ llist_del(&this->m_trx_list.list);
+
+ llist_add(&this->m_trx_list.list, &res->trx->dl_tbfs);
+
+ this->trx = res->trx;
+ this->upgrade_to_multislot = res->upgrade_to_multislot;
+
+ for (ts = 0; ts < ARRAY_SIZE(trx->pdch); ts++) {
+ struct gprs_rlcmac_pdch *pdch = &trx->pdch[ts];
+ OSMO_ASSERT(!this->pdch[pdch->ts_no]);
+ if (!(res->ass_slots_mask & (1 << ts)))
+ continue;
+ LOGPTBFDL(this, LOGL_DEBUG, "Assigning TS=%u TFI=%d\n",
+ ts, res->tfi);
+
+ this->m_tfi = res->tfi;
+
+ this->pdch[pdch->ts_no] = pdch;
+ pdch->attach_tbf(this);
+ }
+
+ /* assign initial control ts */
+ tbf_assign_control_ts(this);
+
+ LOGPTBF(this, LOGL_INFO,
+ "Allocated: trx = %d, ul_slots = %02x, dl_slots = %02x\n",
+ this->trx->trx_no, ul_slots(), dl_slots());
+
+ if (tbf_is_egprs_enabled(this))
+ this->set_window_size();
+
+ tbf_update_state_fsm_name(this);
}
-/* depending on the current TBF, we assign on PACCH or AGCH */
-void gprs_rlcmac_dl_tbf::trigger_ass(struct gprs_rlcmac_tbf *old_tbf)
+void dl_tbf_apply_allocated_resources(struct gprs_rlcmac_dl_tbf *dl_tbf, const struct alloc_resources_res *res)
{
- uint16_t pgroup;
+ dl_tbf->apply_allocated_resources(res);
+}
+
+/* old_tbf (UL TBF or DL TBF) will send a Pkt Dl Ass on PACCH to assign tbf.
+ * Note: It is possible that "tbf == old_tbf" if the TBF is being updated. This can
+ * happen when we first assign over PCH (only single slot is possible) and we want
+ * to upgrade the DL-TBF to be multislot. See code calling tbf_update() for more
+ * information.
+ */
+void dl_tbf_trigger_ass_on_pacch(struct gprs_rlcmac_dl_tbf *tbf, struct gprs_rlcmac_tbf *old_tbf)
+{
+ OSMO_ASSERT(tbf);
+ OSMO_ASSERT(old_tbf);
/* stop pending timer */
- stop_timers("assignment (DL-TBF)");
+ tbf_stop_timers(tbf, "DL assignment (PACCH)");
- /* check for downlink tbf: */
- if (old_tbf) {
- LOGPTBFDL(this, LOGL_DEBUG, "Send downlink assignment on PACCH, because %s exists\n", old_tbf->name());
- osmo_fsm_inst_dispatch(old_tbf->dl_ass_fsm.fi, TBF_DL_ASS_EV_SCHED_ASS, NULL);
+ LOGPTBFDL(tbf, LOGL_DEBUG, "Send downlink assignment on PACCH, because %s exists\n", old_tbf->name());
+ osmo_fsm_inst_dispatch(old_tbf->dl_ass_fsm.fi, TBF_DL_ASS_EV_SCHED_ASS, NULL);
- /* change state */
- osmo_fsm_inst_dispatch(this->state_fsm.fi, TBF_EV_ASSIGN_ADD_PACCH, NULL);
- } else {
- LOGPTBFDL(this, LOGL_DEBUG, "Send downlink assignment on PCH, no TBF exist (IMSI=%s)\n",
- imsi());
+ /* change state */
+ osmo_fsm_inst_dispatch(tbf->state_fi, TBF_EV_ASSIGN_ADD_PACCH, NULL);
+
+}
+
+void dl_tbf_trigger_ass_on_pch(struct gprs_rlcmac_dl_tbf *tbf)
+{
+ /* stop pending timer */
+ struct GprsMs *ms = tbf_ms(tbf);
+
+ tbf_stop_timers(tbf, "DL assignment (PCH)");
+
+ LOGPTBFDL(tbf, LOGL_DEBUG, "Send downlink assignment on PCH, no TBF exist (IMSI=%s)\n", ms_imsi(ms));
+
+ /* change state */
+ osmo_fsm_inst_dispatch(tbf->state_fi, TBF_EV_ASSIGN_ADD_CCCH, NULL);
+}
+
+int dl_tbf_upgrade_to_multislot(struct gprs_rlcmac_dl_tbf *dl_tbf)
+{
+ int rc;
+ struct gprs_rlcmac_tbf *tbf = dl_tbf_as_tbf(dl_tbf);
+ struct gprs_rlcmac_trx *trx = tbf_get_trx(dl_tbf);
+ struct gprs_rlcmac_bts *bts = trx->bts;
+ struct GprsMs *ms = tbf->ms();
- /* change state */
- osmo_fsm_inst_dispatch(this->state_fsm.fi, TBF_EV_ASSIGN_ADD_CCCH, NULL);
+ LOGPTBFDL(dl_tbf, LOGL_DEBUG, "Upgrade to multislot\n");
- /* send immediate assignment */
- if ((pgroup = imsi2paging_group(imsi())) > 999)
- LOGPTBFDL(this, LOGL_ERROR, "IMSI to paging group failed! (%s)\n", imsi());
- bts_snd_dl_ass(bts, this, pgroup);
+ tbf_unlink_pdch(dl_tbf);
+
+ const struct alloc_resources_req req = {
+ .bts = bts,
+ .ms = ms,
+ .direction = tbf_direction(tbf),
+ .single = false,
+ .use_trx = -1,
+ };
+ struct alloc_resources_res res = {};
+
+ rc = the_pcu->alloc_algorithm(&req, &res);
+ /* if no resource */
+ if (rc < 0) {
+ LOGPTBFDL(dl_tbf, LOGL_ERROR, "No resources allocated during upgrade to multislot!\n");
+ bts_do_rate_ctr_inc(bts, CTR_TBF_ALLOC_FAIL);
+ return rc;
}
+
+ /* Update MS, really allocate the resources */
+ if (res.reserved_ul_slots != ms_reserved_ul_slots(ms) ||
+ res.reserved_dl_slots != ms_reserved_dl_slots(ms)) {
+ /* The reserved slots have changed, update the MS */
+ ms_set_reserved_slots(ms, res.trx, res.reserved_ul_slots, res.reserved_dl_slots);
+ }
+ ms_set_first_common_ts(ms, res.first_common_ts);
+
+ /* Apply allocated resources to TBF: */
+ dl_tbf_apply_allocated_resources(dl_tbf, &res);
+
+ /* Note: No need to call ms_attach_tbf(), tbf is already attached to the MS */
+
+ /* Now trigger the assignment using the pre-existing TBF: */
+ dl_tbf_trigger_ass_on_pacch(dl_tbf, tbf);
+ return 0;
}
void gprs_rlcmac_dl_tbf::schedule_next_frame()
@@ -632,13 +544,13 @@ void gprs_rlcmac_dl_tbf::schedule_next_frame()
return;
/* dequeue next LLC frame, if any */
- msg = llc_dequeue(bts->pcu->bssgp.bctx);
+ msg = llc_queue_dequeue(llc_queue(), &m_llc.prio, &m_llc.meta_info);
if (!msg)
return;
LOGPTBFDL(this, LOGL_DEBUG, "Dequeue next LLC (len=%d)\n", msg->len);
- m_llc.put_frame(msg->data, msg->len);
+ llc_put_frame(&m_llc, msg->data, msg->len);
bts_do_rate_ctr_inc(bts, CTR_LLC_FRAME_SCHED);
msgb_free(msg);
m_last_dl_drained_fn = -1;
@@ -710,7 +622,7 @@ int gprs_rlcmac_dl_tbf::create_new_bsn(const uint32_t fn, enum CodingScheme cs)
is_final = llc_queue_size(llc_queue()) == 0 && !keep_open(fn);
if (is_final) {
rdbi->cv = 0;
- osmo_fsm_inst_dispatch(this->state_fsm.fi, TBF_EV_LAST_DL_DATA_SENT, NULL);
+ osmo_fsm_inst_dispatch(this->state_fi, TBF_EV_LAST_DL_DATA_SENT, NULL);
}
if (mcs_is_edge(cs)) {
@@ -730,7 +642,7 @@ int gprs_rlcmac_dl_tbf::create_new_bsn(const uint32_t fn, enum CodingScheme cs)
* "Delayed release of downlink Temporary Block Flow" */
/* A header will need to by added, so we just need
* space-1 octets */
- m_llc.put_dummy_frame(space - 1);
+ llc_put_dummy_frame(&m_llc, space - 1);
LOGPTBFDL(this, LOGL_DEBUG,
"Empty chunk, added LLC dummy command of size %d, drained_since=%d\n",
@@ -751,11 +663,20 @@ int gprs_rlcmac_dl_tbf::create_new_bsn(const uint32_t fn, enum CodingScheme cs)
LOGPTBFDL(this, LOGL_DEBUG, "Complete DL frame, len=%d\n", llc_frame_length(&m_llc));
gprs_rlcmac_dl_bw(this, llc_frame_length(&m_llc));
bts_do_rate_ctr_add(bts, CTR_LLC_DL_BYTES, llc_frame_length(&m_llc));
- m_llc.reset();
+
+ /* Keep transmitted LLC PDUs until first ACK to avoid lossing them if MS is not there. */
+ if (!this->m_first_dl_ack_rcvd) {
+ struct gprs_dl_llc_llist_item *llc_it = talloc(this, struct gprs_dl_llc_llist_item);
+ memcpy(&llc_it->llc, &m_llc, sizeof(llc_it->llc));
+ /* Prepend to list to store them in inverse order of transmission, see
+ * dl_tbf_copy_unacked_pdus_to_llc_queue() for the complete picture. */
+ llist_add(&llc_it->list, &this->tx_llc_until_first_dl_ack_rcvd);
+ }
+ llc_reset(&m_llc);
if (is_final) {
request_dl_ack();
- osmo_fsm_inst_dispatch(this->state_fsm.fi, TBF_EV_LAST_DL_DATA_SENT, NULL);
+ osmo_fsm_inst_dispatch(this->state_fi, TBF_EV_LAST_DL_DATA_SENT, NULL);
}
/* dequeue next LLC frame, if any */
@@ -773,7 +694,7 @@ int gprs_rlcmac_dl_tbf::create_new_bsn(const uint32_t fn, enum CodingScheme cs)
}
struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(
- const uint32_t fn, const uint8_t ts,
+ const uint32_t fn, const struct gprs_rlcmac_pdch *pdch,
int index, int index2)
{
uint8_t *msg_data;
@@ -940,13 +861,14 @@ struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(
m_last_dl_poll_fn = fn;
/* poll after POLL_ACK_AFTER_FRAMES frames, or when final block is tx or
- * when last polled DL ACK/NACK was lost. */
- if (need_poll_for_dl_ack_nack()) {
+ * when last polled DL ACK/NACK was lost. Always do so in the control TS. */
+ if (tbf_is_control_ts(dl_tbf_as_tbf(this), pdch) &&
+ need_poll_for_dl_ack_nack()) {
if (m_dl_ack_requested) {
LOGPTBFDL(this, LOGL_DEBUG,
"Scheduling Ack/Nack polling, because it was requested explicitly "
"(e.g. first final block sent).\n");
- } else if (state_fsm.state_flags & (1 << GPRS_RLCMAC_FLAG_TO_DL_ACK)) {
+ } else if (m_last_dl_poll_ack_lost) {
LOGPTBFDL(this, LOGL_DEBUG,
"Scheduling Ack/Nack polling, because polling timed out.\n");
} else {
@@ -955,22 +877,22 @@ struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(
POLL_ACK_AFTER_FRAMES);
}
- rc = check_polling(fn, ts, &new_poll_fn, &rrbp);
+ rc = tbf_check_polling(this, pdch, fn, &new_poll_fn, &rrbp);
if (rc >= 0) {
- set_polling(new_poll_fn, ts, PDCH_ULC_POLL_DL_ACK);
+ tbf_set_polling(this, pdch, new_poll_fn, PDCH_ULC_POLL_DL_ACK);
LOGPTBFDL(this, LOGL_DEBUG,
"Scheduled DL Acknowledgement polling on PACCH (FN=%d, TS=%d)\n",
- new_poll_fn, ts);
+ new_poll_fn, pdch->ts_no);
m_tx_counter = 0;
/* start timer whenever we send the final block */
if (is_final)
T_START(this, T3191, 3191, "final block (DL-TBF)", true);
- state_fsm.state_flags &= ~(1 << GPRS_RLCMAC_FLAG_TO_DL_ACK); /* clear poll timeout flag */
-
/* Clear request flag */
m_dl_ack_requested = false;
+ /* clear poll timeout flag */
+ m_last_dl_poll_ack_lost = false;
/* set polling in header */
rlc.rrbp = rrbp;
@@ -980,7 +902,7 @@ struct msgb *gprs_rlcmac_dl_tbf::create_dl_acked_block(
LOGPTBFDL(this, LOGL_INFO,
"Scheduled Ack/Nack polling on FN=%d, TS=%d\n",
- new_poll_fn, ts);
+ new_poll_fn, pdch->ts_no);
}
}
@@ -1142,8 +1064,7 @@ int gprs_rlcmac_dl_tbf::update_window(unsigned first_bsn,
int gprs_rlcmac_dl_tbf::rcvd_dl_final_ack()
{
uint16_t received;
-
- osmo_fsm_inst_dispatch(this->state_fsm.fi, TBF_EV_FINAL_ACK_RECVD, NULL);
+ int rc;
/* range V(A)..V(S)-1 */
received = m_window.count_unacked();
@@ -1152,12 +1073,9 @@ int gprs_rlcmac_dl_tbf::rcvd_dl_final_ack()
m_tx_counter = 0;
m_window.reset();
- /* check for LLC PDU in the LLC Queue */
- if (llc_queue_size(llc_queue()) > 0)
- /* we have more data so we will re-use this tbf */
- establish_dl_tbf_on_pacch();
+ rc = osmo_fsm_inst_dispatch(this->state_fi, TBF_EV_FINAL_ACK_RECVD, NULL);
- return 0;
+ return rc;
}
int gprs_rlcmac_dl_tbf::rcvd_dl_ack(bool final_ack, unsigned first_bsn,
@@ -1166,8 +1084,16 @@ int gprs_rlcmac_dl_tbf::rcvd_dl_ack(bool final_ack, unsigned first_bsn,
int rc;
LOGPTBFDL(this, LOGL_DEBUG, "downlink acknowledge\n");
- state_fsm.state_flags |= (1 << GPRS_RLCMAC_FLAG_DL_ACK);
- state_fsm.state_flags &= ~(1 << GPRS_RLCMAC_FLAG_TO_DL_ACK);
+ if (m_first_dl_ack_rcvd == false) {
+ /* MS is there, free temporarily stored transmitted LLC PDUs */
+ struct gprs_dl_llc_llist_item *llc_it;
+ while ((llc_it = llist_first_entry_or_null(&this->tx_llc_until_first_dl_ack_rcvd, struct gprs_dl_llc_llist_item, list))) {
+ llist_del(&llc_it->list);
+ talloc_free(llc_it);
+ }
+ m_first_dl_ack_rcvd = true;
+ }
+ m_last_dl_poll_ack_lost = false;
/* reset N3105 */
n_reset(N3105);
@@ -1196,12 +1122,48 @@ void gprs_rlcmac_dl_tbf::request_dl_ack()
m_dl_ack_requested = true;
}
+void dl_tbf_request_dl_ack(struct gprs_rlcmac_dl_tbf *dl_tbf) {
+ dl_tbf->request_dl_ack();
+}
+
+bool dl_tbf_first_dl_ack_rcvd(const struct gprs_rlcmac_dl_tbf *tbf)
+{
+ return tbf->m_first_dl_ack_rcvd;
+}
+
+/* Copy back to GprsMs' llc_queue the LLC PDUs previously dequeued and never
+ * fully ACKED at the MS side.
+ * FIXME: For now, only blocks transmitted and without first ever DL ACK are
+ * copied, because we have no way yet to track LLC PDUs once they are converted
+ * to RLC blocks. This is however enough to cover the case where a DL TBF is
+ * assigned over PCH and the MS never answers.
+ */
+void dl_tbf_copy_unacked_pdus_to_llc_queue(struct gprs_rlcmac_dl_tbf *tbf)
+{
+ struct GprsMs *ms = tbf_ms(dl_tbf_as_tbf(tbf));
+ struct gprs_dl_llc_llist_item *llc_it;
+
+ /* If we have LLC PDU still being transmitted, prepend it first to the queue: */
+ if (llc_frame_length(&tbf->m_llc) > 0)
+ llc_queue_merge_prepend(&ms->llc_queue, &tbf->m_llc);
+
+ /* Iterate over the list of totally transmitted LLC PDUs and merge them
+ * into the queue. The items in the list are in inverse order of
+ * transmission, hence when popping from here and enqueueing (prepending)
+ * back to the llc_queue it ends up in the exact same initial order. */
+ while ((llc_it = llist_first_entry_or_null(&tbf->tx_llc_until_first_dl_ack_rcvd, struct gprs_dl_llc_llist_item, list))) {
+ llist_del(&llc_it->list);
+ llc_queue_merge_prepend(&ms->llc_queue, &llc_it->llc);
+ talloc_free(llc_it);
+ }
+}
+
/* Does this DL TBF require to poll the MS for DL ACK/NACK? */
bool gprs_rlcmac_dl_tbf::need_poll_for_dl_ack_nack() const
{
/* poll after POLL_ACK_AFTER_FRAMES frames, or when final block is tx or
* when last polled DL ACK/NACK was lost. */
- return state_fsm.state_flags & (1 << GPRS_RLCMAC_FLAG_TO_DL_ACK) ||
+ return m_last_dl_poll_ack_lost ||
m_tx_counter >= POLL_ACK_AFTER_FRAMES ||
m_dl_ack_requested;
}
@@ -1442,7 +1404,7 @@ void gprs_rlcmac_dl_tbf::update_coding_scheme_counter_dl(enum CodingScheme cs)
}
}
-struct gprs_rlcmac_dl_tbf *as_dl_tbf(struct gprs_rlcmac_tbf *tbf)
+struct gprs_rlcmac_dl_tbf *tbf_as_dl_tbf(struct gprs_rlcmac_tbf *tbf)
{
if (tbf && tbf->direction == GPRS_RLCMAC_DL_TBF)
return static_cast<gprs_rlcmac_dl_tbf *>(tbf);
@@ -1450,7 +1412,10 @@ struct gprs_rlcmac_dl_tbf *as_dl_tbf(struct gprs_rlcmac_tbf *tbf)
return NULL;
}
-void tbf_dl_trigger_ass(struct gprs_rlcmac_dl_tbf *tbf, struct gprs_rlcmac_tbf *old_tbf)
+const struct gprs_rlcmac_dl_tbf *tbf_as_dl_tbf_const(const struct gprs_rlcmac_tbf *tbf)
{
- return tbf->trigger_ass(old_tbf);
+ if (tbf && tbf->direction == GPRS_RLCMAC_DL_TBF)
+ return static_cast<const gprs_rlcmac_dl_tbf *>(tbf);
+ else
+ return NULL;
}