struct rt_cork rt_cork;
-/* Data structures for export journal */
-#define RT_PENDING_EXPORT_ITEMS (page_size - sizeof(struct rt_export_block)) / sizeof(struct rt_pending_export)
-
struct rt_export_block {
- node n;
- _Atomic u32 end;
- _Atomic _Bool not_last;
+ struct lfjour_block lb;
struct rt_pending_export export[];
};
+
+/* Data structures for export journal */
+#define RT_PENDING_EXPORT_ITEMS (page_size - sizeof(struct rt_export_block)) / sizeof(struct rt_pending_export)
+
static void rt_free_hostcache(struct rtable_private *tab);
static void rt_update_hostcache(void *tab);
static void rt_next_hop_update(void *_tab);
static inline void rt_next_hop_resolve_rte(rte *r);
static inline void rt_flowspec_resolve_rte(rte *r, struct channel *c);
static void rt_refresh_trace(struct rtable_private *tab, struct rt_import_hook *ih, const char *msg);
-static inline void rt_prune_table(struct rtable_private *tab);
static void rt_kick_prune_timer(struct rtable_private *tab);
+static void rt_prune_table(void *_tab);
static void rt_feed_by_fib(void *);
static void rt_feed_by_trie(void *);
static void rt_feed_equal(void *);
static void rt_shutdown(void *);
static void rt_delete(void *);
-static void rt_export_used(struct rt_exporter *, const char *, const char *);
-static void rt_export_cleanup(struct rtable_private *tab);
-
int rte_same(const rte *x, const rte *y);
const char *rt_import_state_name_array[TIS_MAX] = {
return NULL;
}
-static struct rt_pending_export * rt_next_export_fast(struct rt_pending_export *last);
-static int
+static void
rte_export(struct rt_export_hook *hook, struct rt_pending_export *rpe)
{
- rtable *tab = RT_PUB(SKIP_BACK(struct rtable_private, exporter, hook->table));
+ /* Seen already? */
if (bmap_test(&hook->seq_map, rpe->seq))
- goto ignore; /* Seen already */
+ return;
const net_addr *n = rpe->new_best ? rpe->new_best->net : rpe->old_best->net;
/* Check export eligibility of this net */
if (!rt_prefilter_net(&hook->req->prefilter, n))
- goto ignore;
+ return;
if (hook->req->prefilter.mode == TE_ADDR_FOR)
bug("Continuos export of best prefix match not implemented yet.");
uint count = 0;
const rte **feed = NULL;
- const struct netindex *i = SKIP_BACK(struct netindex, addr, (net_addr (*)[0]) n);
-
- RT_LOCKED(tab, tp)
+ RT_LOCKED(hook->tab, tp)
{
+ const struct netindex *i = SKIP_BACK(struct netindex, addr, (net_addr (*)[0]) n);
+
ASSERT_DIE(i->index < tp->routes_block_size);
struct network *net = &tp->routes[i->index];
last = net->last;
}
else
bug("Export request must always provide an export method");
-
-ignore:
- /* Get the next export if exists */
- hook->rpe_next = rt_next_export_fast(rpe);
-
- /* The last block may be available to free */
- int used = (PAGE_HEAD(hook->rpe_next) != PAGE_HEAD(rpe));
-
- /* Releasing this export for cleanup routine */
- DBG("store hook=%p last_export=%p seq=%lu\n", hook, rpe, rpe->seq);
- atomic_store_explicit(&hook->last_export, rpe, memory_order_release);
-
- return used;
}
/**
if (old_best_valid)
old_best->sender->stats.pref--;
- if (EMPTY_LIST(tab->exporter.hooks) && EMPTY_LIST(tab->exporter.pending))
- {
- /* No export hook and no pending exports to cleanup. We may free the route immediately. */
- if (!old)
- return;
+ struct rt_pending_export *rpe = SKIP_BACK(struct rt_pending_export, li, lfjour_push_prepare(&tab->journal));
- hmap_clear(&tab->id_map, old->id);
- rte_free(SKIP_BACK(struct rte_storage, rte, old), tab);
- return;
- }
-
- /* Get the pending export structure */
- struct rt_export_block *rpeb = NULL, *rpebsnl = NULL;
- u32 end = 0;
-
- if (!EMPTY_LIST(tab->exporter.pending))
+ if (!rpe)
{
- rpeb = TAIL(tab->exporter.pending);
- end = atomic_load_explicit(&rpeb->end, memory_order_relaxed);
- if (end >= RT_PENDING_EXPORT_ITEMS)
+ rt_trace(tab, D_ROUTES, "Not announcing %N, "
+ "new=%p id %u from %s, "
+ "old=%p id %u from %s, "
+ "new_best=%p id %u, "
+ "old_best=%p id %u (no exporter present)",
+ i->addr,
+ new, new ? new->id : 0, new ? new->sender->req->name : NULL,
+ old, old ? old->id : 0, old ? old->sender->req->name : NULL,
+ new_best, new_best ? new_best->id : 0,
+ old_best, old_best ? old_best->id : 0);
+ /* Not announcing, can free old route immediately */
+ if (old)
{
- ASSERT_DIE(end == RT_PENDING_EXPORT_ITEMS);
- rpebsnl = rpeb;
-
- rpeb = NULL;
- end = 0;
+ hmap_clear(&tab->id_map, old->id);
+ rte_free(SKIP_BACK(struct rte_storage, rte, old), tab);
}
+ return;
}
- if (!rpeb)
- {
- rpeb = alloc_page();
- *rpeb = (struct rt_export_block) {};
- add_tail(&tab->exporter.pending, &rpeb->n);
- }
-
- /* Fill the pending export */
- struct rt_pending_export *rpe = &rpeb->export[rpeb->end];
- *rpe = (struct rt_pending_export) {
- .new = new,
- .new_best = new_best,
- .old = old,
- .old_best = old_best,
- .seq = tab->exporter.next_seq++,
- };
-
rt_trace(tab, D_ROUTES, "Announcing %N, "
"new=%p id %u from %s, "
"old=%p id %u from %s, "
old, old ? old->id : 0, old ? old->sender->req->name : NULL,
new_best, new_best ? new_best->id : 0,
old_best, old_best ? old_best->id : 0,
- rpe->seq);
+ rpe->li.seq);
- ASSERT_DIE(atomic_fetch_add_explicit(&rpeb->end, 1, memory_order_release) == end);
+ *rpe = (struct rt_pending_export) {
+ .li = rpe->li, /* Keep the item's internal state */
+ .new = new,
+ .new_best = new_best,
+ .old = old,
+ .old_best = old_best,
+ };
- if (rpebsnl)
- {
- _Bool f = 0;
- ASSERT_DIE(atomic_compare_exchange_strong_explicit(&rpebsnl->not_last, &f, 1,
- memory_order_release, memory_order_relaxed));
- }
+ lfjour_push_commit(&tab->journal);
/* Append to the same-network squasher list */
if (net->last)
if (!net->first)
net->first = rpe;
- if (tab->exporter.first == NULL)
- tab->exporter.first = rpe;
-
rt_check_cork_high(tab);
}
-static struct rt_pending_export *
-rt_next_export_fast(struct rt_pending_export *last)
+static inline void
+rt_send_export_event(struct rt_export_hook *hook)
{
- /* Get the whole export block and find our position in there. */
- struct rt_export_block *rpeb = PAGE_HEAD(last);
- u32 pos = (last - &rpeb->export[0]);
- u32 end = atomic_load_explicit(&rpeb->end, memory_order_acquire);
- ASSERT_DIE(pos < end);
-
- /* Next is in the same block. */
- if (++pos < end)
- return &rpeb->export[pos];
-
- /* There is another block. */
- if (atomic_load_explicit(&rpeb->not_last, memory_order_acquire))
- {
- /* This is OK to do non-atomically because of the not_last flag. */
- rpeb = NODE_NEXT(rpeb);
- return &rpeb->export[0];
- }
-
- /* There is nothing more. */
- return NULL;
+ ev_send(hook->req->list, hook->event);
}
-static struct rt_pending_export *
-rt_next_export(struct rt_export_hook *hook, struct rt_exporter *tab)
+static void
+rt_cleanup_export(struct lfjour *j, struct lfjour_item *i)
{
- ASSERT_DIE(RT_IS_LOCKED(SKIP_BACK(struct rtable_private, exporter, tab)));
+ struct rtable_private *tab = SKIP_BACK(struct rtable_private, journal, j);
+ struct rt_pending_export *rpe = SKIP_BACK(struct rt_pending_export, li, i);
- /* As the table is locked, it is safe to reload the last export pointer */
- struct rt_pending_export *last = atomic_load_explicit(&hook->last_export, memory_order_acquire);
+ /* Unlink this export from struct network */
+ ASSERT_DIE(rpe->new || rpe->old);
+ const net_addr *n = rpe->new ?
+ rpe->new->net :
+ rpe->old->net;
+ struct netindex *ni = NET_TO_INDEX(n);
+ ASSERT_DIE(ni->index < tab->routes_block_size);
+ net *net = &tab->routes[ni->index];
- /* It is still valid, let's reuse it */
- if (last)
- return rt_next_export_fast(last);
+ ASSERT_DIE(net->first == rpe);
- /* No, therefore we must process the table's first pending export */
+ if (rpe == net->last)
+ /* The only export here */
+ net->last = net->first = NULL;
else
- return tab->first;
-}
-
-static inline void
-rt_send_export_event(struct rt_export_hook *hook)
-{
- ev_send(hook->req->list, &hook->event);
-}
+ /* First is now the next one */
+ net->first = atomic_load_explicit(&rpe->next, memory_order_relaxed);
-static void
-rt_announce_exports(struct settle *s)
-{
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, export_settle, s)), tab)
- if (!EMPTY_LIST(tab->exporter.pending))
- {
- struct rt_export_hook *c; node *n;
- WALK_LIST2(c, n, tab->exporter.hooks, n)
- {
- if (atomic_load_explicit(&c->export_state, memory_order_acquire) != TES_READY)
- continue;
+ if (rpe->old)
+ {
+ hmap_clear(&tab->id_map, rpe->old->id);
+ rte_free(SKIP_BACK(struct rte_storage, rte, rpe->old), tab);
+ }
- rt_send_export_event(c);
- }
- }
+ if (!net->routes && !net->first)
+ tab->gc_counter++;
}
static void
-rt_kick_export_settle(struct rtable_private *tab)
+rt_import_cleared(void *_ih)
{
- tab->export_settle.cf = tab->rr_counter ? tab->config->export_rr_settle : tab->config->export_settle;
- settle_kick(&tab->export_settle, tab->loop);
-}
+ struct rt_import_hook *hook = _ih;
-static void
-rt_import_announce_exports(void *_hook)
-{
- struct rt_import_hook *hook = _hook;
- if (hook->import_state == TIS_CLEARED)
- {
- void (*stopped)(struct rt_import_request *) = hook->stopped;
- struct rt_import_request *req = hook->req;
+ ASSERT_DIE(hook->import_state == TIS_CLEARED);
- RT_LOCKED(hook->table, tab)
- {
- req->hook = NULL;
+ /* Local copy of the otherwise freed callback data */
+ void (*stopped)(struct rt_import_request *) = hook->stopped;
+ struct rt_import_request *req = hook->req;
- rt_trace(tab, D_EVENTS, "Hook %s stopped", req->name);
- rem_node(&hook->n);
- mb_free(hook);
- rt_unlock_table(tab);
- }
+ /* Finally uncouple from the table */
+ RT_LOCKED(hook->table, tab)
+ {
+ req->hook = NULL;
- stopped(req);
- return;
+ rt_trace(tab, D_EVENTS, "Hook %s stopped", req->name);
+ rem_node(&hook->n);
+ mb_free(hook);
+ rt_unlock_table(tab);
}
- rt_trace(hook->table, D_EVENTS, "Announcing exports after imports from %s", hook->req->name);
- birdloop_flag(hook->table->loop, RTF_EXPORT);
+ /* And call the callback */
+ stopped(req);
}
-static struct rt_pending_export *
-rt_last_export(struct rt_exporter *tab)
+static void
+rt_cleanup_done(struct lfjour *j, u64 begin_seq, u64 end_seq)
{
- struct rt_pending_export *rpe = NULL;
+ struct rtable_private *tab = SKIP_BACK(struct rtable_private, journal, j);
+ ASSERT_DIE(DG_IS_LOCKED(tab->lock.rtable));
- if (!EMPTY_LIST(tab->pending))
- {
- /* We'll continue processing exports from this export on */
- struct rt_export_block *reb = TAIL(tab->pending);
- ASSERT_DIE(reb->end);
- rpe = &reb->export[reb->end - 1];
- }
+ if (~end_seq)
+ rt_trace(tab, D_STATES, "Export cleanup done on seq %lu to %lu", begin_seq, end_seq);
+ else
+ rt_trace(tab, D_STATES, "Export cleanup complete (begin seq %lu)", begin_seq);
+
+ rt_check_cork_low(tab);
- return rpe;
+ struct rt_import_hook *ih; node *x, *n;
+ uint cleared_counter = 0;
+ if (tab->wait_counter)
+ WALK_LIST2_DELSAFE(ih, n, x, tab->imports, n)
+ if (ih->import_state == TIS_WAITING)
+ if (end_seq >= ih->flush_seq)
+ {
+ ih->import_state = TIS_CLEARED;
+ tab->wait_counter--;
+ cleared_counter++;
+
+ ih->cleanup_event = (event) {
+ .hook = rt_import_cleared,
+ .data = ih,
+ };
+ ev_send_loop(ih->req->loop, &ih->cleanup_event);
+ }
+
+ if (!EMPTY_LIST(tab->imports) &&
+ (tab->gc_counter >= tab->config->gc_threshold))
+ rt_kick_prune_timer(tab);
}
#define RT_EXPORT_BULK 1024
rt_export_hook(void *_data)
{
struct rt_export_hook *c = _data;
- rtable *tab = SKIP_BACK(rtable, priv.exporter, c->table);
+ struct lfjour_recipient *r = &c->recipient;
ASSERT_DIE(atomic_load_explicit(&c->export_state, memory_order_relaxed) == TES_READY);
- if (!c->rpe_next)
- RT_LOCKED(tab, tp)
- {
- c->rpe_next = rt_next_export(c, c->table);
-
- if (!c->rpe_next)
- {
- rt_export_used(c->table, c->req->name, "done exporting");
- return;
- }
- }
-
- int used = 0;
- int no_next = 0;
-
/* Process the export */
for (uint i=0; i<RT_EXPORT_BULK; i++)
{
- used += rte_export(c, c->rpe_next);
+ /* Get the next export if exists */
+ struct lfjour_item *li = lfjour_get(r);
- if (!c->rpe_next)
- {
- no_next = 1;
- break;
- }
+ /* Stop exporting if no export is available */
+ if (!li)
+ return;
+
+ /* Process sequence number reset event */
+ if (lfjour_reset_seqno(r))
+ bmap_reset(&c->seq_map, 16);
+
+ /* Process the export */
+ rte_export(c, SKIP_BACK(struct rt_pending_export, li, li));
+
+ /* And release the export */
+ lfjour_release(r);
}
+ /*
+ * is this actually needed?
if (used)
RT_LOCKED(tab, t)
if (no_next || t->cork_active)
rt_export_used(c->table, c->req->name, no_next ? "finished export bulk" : "cork active");
+ */
+ /* Request continuation */
rt_send_export_event(c);
}
static inline int rte_is_ok(const rte *e) { return e && !rte_is_filtered(e); }
-static int
+static void
rte_recalculate(struct rtable_private *table, struct rt_import_hook *c, struct netindex *i, net *net, rte *new, struct rte_src *src)
{
struct rt_import_request *req = c->req;
/* We need to free the already stored route here before returning */
rte_free(new_stored, table);
- return 0;
+ return;
}
*before_old = (*before_old)->next;
if (!old && !new)
{
stats->withdraws_ignored++;
- return 0;
+ return;
}
/* If rejected by import limit, we need to pretend there is no route */
RTE_OR_NULL(new_stored), RTE_OR_NULL(old_stored),
RTE_OR_NULL(net->routes), RTE_OR_NULL(old_best_stored));
- return 1;
+ return;
}
int
int new_in = new && !rte_is_filtered(new);
int old_in = old && !rte_is_filtered(old);
-
+
int verdict = 1;
if (new_in && !old_in)
}
/* Recalculate the best route */
- if (rte_recalculate(tab, hook, i, nn, new, src))
- ev_send(req->list, &hook->announce_event);
+ rte_recalculate(tab, hook, i, nn, new, src);
}
}
struct rt_export_hook *hook = hh;
struct rt_export_request *req = hook->req;
void (*stopped)(struct rt_export_request *) = hook->stopped;
- rtable *t = SKIP_BACK(rtable, priv.exporter, hook->table);
+ rtable *t = hook->tab;
+ /* Drop the hook */
RT_LOCKED(t, tab)
{
- DBG("Export hook %p in table %s finished uc=%u\n", hook, tab->name, tab->use_count);
+ /* Unlink from the table */
+ if (lfjour_of_recipient(&hook->recipient))
+ lfjour_unregister(&hook->recipient);
- /* Drop pending exports */
- rt_export_used(&tab->exporter, hook->req->name, "stopped");
+ DBG("Export hook %p in table %s finished uc=%u\n", hook, tab->name, tab->use_count);
- /* Do the common code; this frees the hook */
- rt_export_stopped(hook);
+ /* Free the hook */
+ rp_free(hook->pool);
}
- /* Report the channel as stopped. */
+ /* Inform the stopper */
CALL(stopped, req);
- /* Unlock the table; this may free it */
+ /* Unlock the table */
rt_unlock_table(t);
}
-void
-rt_export_stopped(struct rt_export_hook *hook)
-{
- /* Unlink from the request */
- hook->req->hook = NULL;
-
- /* Unlist */
- rem_node(&hook->n);
-
- /* Free the hook itself together with its pool */
- rp_free(hook->pool);
-}
-
static inline void
rt_set_import_state(struct rt_import_hook *hook, u8 state)
{
struct rt_import_hook *hook = req->hook = mb_allocz(tab->rp, sizeof(struct rt_import_hook));
- hook->announce_event = (event) { .hook = rt_import_announce_exports, .data = hook };
-
DBG("Lock table %s for import %p req=%p uc=%u\n", tab->name, hook, req, tab->use_count);
hook->req = req;
RT_LOCKED(hook->table, tab)
{
- rt_schedule_prune(tab);
rt_set_import_state(hook, TIS_STOP);
hook->stopped = stopped;
tab->rr_counter++;
hook->stale_set = hook->stale_pruned = hook->stale_pruning = hook->stale_valid = 0;
+
+ rt_schedule_prune(tab);
}
}
birdloop_enter(loop);
u8 state;
- switch (state = atomic_load_explicit(&hook->export_state, memory_order_relaxed))
- {
- case TES_HUNGRY:
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, exporter, hook->table)), tab)
- if ((state = atomic_load_explicit(&hook->export_state, memory_order_relaxed)) == TES_HUNGRY)
- rt_table_export_start_feed(tab, hook);
- if (state != TES_STOP)
+ RT_LOCKED(hook->tab, tab)
+ switch (state = atomic_load_explicit(&hook->export_state, memory_order_relaxed))
+ {
+ case TES_HUNGRY:
+ rt_table_export_start_feed(tab, hook);
break;
- /* fall through */
- case TES_STOP:
- rt_stop_export_common(hook);
- break;
- default:
- bug("Uncorking a table export in a strange state: %u", state);
- }
+ case TES_STOP:
+ hook->event->hook = rt_table_export_done;
+ rt_send_export_event(hook);
+ break;
+ default:
+ bug("Uncorking a table export in a strange state: %u", state);
+ }
if (loop != &main_birdloop)
birdloop_leave(loop);
static void
rt_table_export_start_locked(struct rtable_private *tab, struct rt_export_request *req)
{
- struct rt_exporter *re = &tab->exporter;
rt_lock_table(tab);
- req->hook = rt_alloc_export(re, req->pool, sizeof(struct rt_export_hook));
- req->hook->req = req;
-
- struct rt_export_hook *hook = req->hook;
- hook->event = (event) {
- .hook = rt_table_export_uncork,
- .data = hook,
- };
+ pool *p = rp_new(req->pool, req->pool->domain, "Export hook");
+ struct rt_export_hook *hook = req->hook = mb_allocz(p, sizeof(struct rt_export_hook));
+ hook->req = req;
+ hook->tab = RT_PUB(tab);
+ hook->pool = p;
+ atomic_store_explicit(&hook->export_state, TES_DOWN, memory_order_release);
+ hook->event = ev_new_init(p, rt_table_export_uncork, hook);
- if (rt_cork_check(&hook->event))
+ if (rt_cork_check(hook->event))
rt_set_export_state(hook, BIT32_ALL(TES_DOWN), TES_HUNGRY);
else
rt_table_export_start_feed(tab, hook);
static void
rt_table_export_start_feed(struct rtable_private *tab, struct rt_export_hook *hook)
{
- struct rt_exporter *re = &tab->exporter;
struct rt_export_request *req = hook->req;
/* stats zeroed by mb_allocz */
hook->walk_state = mb_allocz(hook->pool, sizeof (struct f_trie_walk_state));
hook->walk_lock = rt_lock_trie(tab);
trie_walk_init(hook->walk_state, tab->trie, req->prefilter.addr);
- hook->event.hook = rt_feed_by_trie;
+ hook->event->hook = rt_feed_by_trie;
hook->walk_last.type = 0;
break;
}
case TE_ADDR_TRIE:
case TE_ADDR_HOOK:
hook->feed_index = 0;
- hook->event.hook = rt_feed_by_fib;
+ hook->event->hook = rt_feed_by_fib;
break;
case TE_ADDR_EQUAL:
- hook->event.hook = rt_feed_equal;
+ hook->event->hook = rt_feed_equal;
break;
case TE_ADDR_FOR:
- hook->event.hook = rt_feed_for;
+ hook->event->hook = rt_feed_for;
break;
default:
DBG("New export hook %p req %p in table %s uc=%u\n", hook, req, tab->name, tab->use_count);
- struct rt_pending_export *rpe = rt_last_export(hook->table);
+ hook->recipient = (struct lfjour_recipient) {
+ .event = hook->event,
+ .target = req->list,
+ };
+ lfjour_register(&tab->journal, &hook->recipient);
+
+ struct rt_pending_export *rpe = SKIP_BACK(struct rt_pending_export, li, atomic_load_explicit(&hook->recipient.last, memory_order_relaxed));
req_trace(req, D_STATES, "Export initialized, last export %p (%lu)", rpe, rpe ? rpe->seq : 0);
- atomic_store_explicit(&hook->last_export, rpe, memory_order_relaxed);
- rt_init_export(re, req->hook);
+ bmap_init(&hook->seq_map, hook->pool, 16);
+
+ /* Regular export */
+ rt_set_export_state(hook, BIT32_ALL(TES_DOWN, TES_HUNGRY), TES_FEEDING);
+ rt_send_export_event(hook);
}
+#if 0
static void
rt_table_export_start(struct rt_exporter *re, struct rt_export_request *req)
{
RT_LOCKED(SKIP_BACK(rtable, priv.exporter, re), tab)
rt_table_export_start_locked(tab, req);
}
-
+#endif
+
void rt_request_export(rtable *t, struct rt_export_request *req)
{
RT_LOCKED(t, tab)
rt_table_export_start_locked(tab, req); /* Is locked inside */
}
-void
-rt_request_export_other(struct rt_exporter *re, struct rt_export_request *req)
-{
- return re->class->start(re, req);
-}
-
-struct rt_export_hook *
-rt_alloc_export(struct rt_exporter *re, pool *pp, uint size)
-{
- pool *p = rp_new(pp, pp->domain, "Export hook");
- struct rt_export_hook *hook = mb_allocz(p, size);
-
- hook->pool = p;
- hook->table = re;
- atomic_store_explicit(&hook->export_state, TES_DOWN, memory_order_release);
-
- hook->n = (node) {};
- add_tail(&re->hooks, &hook->n);
-
- return hook;
-}
-
-void
-rt_init_export(struct rt_exporter *re UNUSED, struct rt_export_hook *hook)
-{
- hook->event.data = hook;
-
- bmap_init(&hook->seq_map, hook->pool, 16);
-
- /* Regular export */
- rt_set_export_state(hook, BIT32_ALL(TES_DOWN, TES_HUNGRY), TES_FEEDING);
- rt_send_export_event(hook);
-}
-
-static int
-rt_table_export_stop_locked(struct rt_export_hook *hook)
+static void
+rt_stop_export_locked(struct rtable_private *tab, struct rt_export_hook *hook)
{
- struct rtable_private *tab = SKIP_BACK(struct rtable_private, exporter, hook->table);
+ struct rt_export_request *req = hook->req;
/* Update export state, get old */
switch (rt_set_export_state(hook, BIT32_ALL(TES_HUNGRY, TES_FEEDING, TES_READY), TES_STOP))
{
+ case TES_STOP:
+ rt_trace(tab, D_EVENTS, "Stopping export hook %s already requested", req->name);
+ return;
+
case TES_HUNGRY:
- rt_trace(tab, D_EVENTS, "Stopping export hook %s must wait for uncorking", hook->req->name);
- return 0;
+ rt_trace(tab, D_EVENTS, "Stopping export hook %s must wait for uncorking", req->name);
+ return;
+
case TES_FEEDING:
- switch (hook->req->prefilter.mode)
+ switch (req->prefilter.mode)
{
case TE_ADDR_IN:
if (hook->walk_lock)
break;
}
break;
-
- case TES_STOP:
- bug("Tried to repeatedly stop the same export hook %s", hook->req->name);
}
- rt_trace(tab, D_EVENTS, "Stopping export hook %s right now", hook->req->name);
- return 1;
-}
-
-static void
-rt_table_export_stop(struct rt_export_hook *hook)
-{
- int ok = 0;
+ rt_trace(tab, D_EVENTS, "Stopping export hook %s right now", req->name);
- rtable *t = SKIP_BACK(rtable, priv.exporter, hook->table);
- if (RT_IS_LOCKED(t))
- ok = rt_table_export_stop_locked(hook);
- else
- RT_LOCKED(t, tab)
- ok = rt_table_export_stop_locked(hook);
+ /* Reset the event as the stopped event */
+ ASSERT_DIE(birdloop_inside(req->list->loop));
+ hook->event->hook = rt_table_export_done;
- if (ok)
- rt_stop_export_common(hook);
+ /* Run the stopped event */
+ rt_send_export_event(hook);
}
void
rt_stop_export(struct rt_export_request *req, void (*stopped)(struct rt_export_request *))
{
ASSERT_DIE(birdloop_inside(req->list->loop));
- ASSERT_DIE(req->hook);
struct rt_export_hook *hook = req->hook;
+ ASSERT_DIE(hook);
- /* Set the stopped callback */
- hook->stopped = stopped;
+ RT_LOCKED(hook->tab, t)
+ {
+ /* Set the stopped callback */
+ hook->stopped = stopped;
- /* Run the stop code. Must:
- * _locked_ update export state to TES_STOP
- * and _unlocked_ call rt_stop_export_common() */
- hook->table->class->stop(hook);
+ /* Do the rest */
+ rt_stop_export_locked(t, hook);
+ }
}
-/* Call this common code from the stop code in table export class */
-void
-rt_stop_export_common(struct rt_export_hook *hook)
-{
- /* Reset the event as the stopped event */
- hook->event.hook = hook->table->class->done;
-
- /* Run the stopped event */
- rt_send_export_event(hook);
-}
/**
* rt_refresh_begin - start a refresh cycle
ih->last_state_change, ih->import_state, ih->stopped);
}
- struct rt_export_hook *eh;
- WALK_LIST(eh, tab->exporter.hooks)
+ WALK_TLIST(lfjour_recipient, r, &tab->journal.recipients)
{
+ struct rt_export_hook *eh = SKIP_BACK(struct rt_export_hook, recipient, r);
eh->req->dump_req(eh->req);
debug(" Export hook %p requested by %p:"
" refeed_pending=%u last_state_change=%t export_state=%u\n",
void
rt_schedule_prune(struct rtable_private *tab)
{
- if (tab->prune_state == 0)
- birdloop_flag(tab->loop, RTF_CLEANUP);
-
/* state change 0->1, 2->3 */
tab->prune_state |= 1;
-}
-
-static void
-rt_export_used(struct rt_exporter *e, const char *who, const char *why)
-{
- struct rtable_private *tab = SKIP_BACK(struct rtable_private, exporter, e);
- ASSERT_DIE(RT_IS_LOCKED(tab));
-
- rt_trace(tab, D_EVENTS, "Export cleanup requested by %s %s", who, why);
-
- if (tab->export_used)
- return;
-
- tab->export_used = 1;
- birdloop_flag(tab->loop, RTF_CLEANUP);
+ ev_send_loop(tab->loop, tab->prune_event);
}
static void
ASSERT_DIE(birdloop_inside(tab->loop));
rt_lock_table(tab);
- if (flags & RTF_EXPORT)
- rt_kick_export_settle(tab);
-
- if (flags & RTF_CLEANUP)
- {
- if (tab->export_used)
- rt_export_cleanup(tab);
-
- if (tab->prune_state)
- rt_prune_table(tab);
- }
-
if (flags & RTF_DELETE)
{
if (tab->hostcache)
- rt_stop_export(&tab->hostcache->req, NULL);
+ rt_stop_export_locked(tab, tab->hostcache->req.hook);
rt_unlock_table(tab);
}
if (!--ln->uc)
{
rt_flowspec_link_rem_node(&t->flowspec_links, ln);
- rt_stop_export(&ln->req, rt_flowspec_link_stopped);
+ ln->req.hook->stopped = rt_flowspec_link_stopped;
+ rt_stop_export_locked(t, ln->req.hook);
}
}
debug("name \"%s\", addr_type=%s, rt_count=%u, use_count=%d\n",
r->name, net_label[r->addr_type], r->rt_count, r->use_count);
+ /* TODO: move this to lfjour */
char x[32];
bsprintf(x, "%%%dspending export %%p\n", indent + 2);
- node *n;
- WALK_LIST(n, r->exporter.pending)
+ WALK_TLIST(lfjour_block, n, &r->journal.pending)
debug(x, "", n);
}
.memsize = NULL,
};
-static const struct rt_exporter_class rt_exporter_class = {
- .start = rt_table_export_start,
- .stop = rt_table_export_stop,
- .done = rt_table_export_done,
-};
-
-void
-rt_exporter_init(struct rt_exporter *e)
-{
- init_list(&e->hooks);
-}
-
static struct idm rtable_idm;
uint rtable_max_id = 0;
t->nhu_event = ev_new_init(p, rt_next_hop_update, t);
t->nhu_uncork_event = ev_new_init(p, rt_nhu_uncork, t);
t->prune_timer = tm_new_init(p, rt_prune_timer, t, 0, 0);
+ t->prune_event = ev_new_init(p, rt_prune_table, t);
t->last_rt_change = t->gc_time = current_time();
- t->export_settle = SETTLE_INIT(&cf->export_settle, rt_announce_exports, NULL);
-
- t->exporter = (struct rt_exporter) {
- .class = &rt_exporter_class,
- .addr_type = t->addr_type,
- .rp = t->rp,
- .next_seq = 1,
- };
-
- rt_exporter_init(&t->exporter);
-
- init_list(&t->exporter.pending);
+ t->journal.loop = t->loop;
+ t->journal.domain = t->lock.rtable;
+ t->journal.item_size = sizeof(struct rt_pending_export);
+ t->journal.item_done = rt_cleanup_export;
+ t->journal.cleanup_done = rt_cleanup_done;
+ lfjour_init(&t->journal, &cf->export_settle);
t->cork_threshold = cf->cork_threshold;
* iteration.
*/
static void
-rt_prune_table(struct rtable_private *tab)
+rt_prune_table(void *_tab)
{
- int limit = 2000;
+ RT_LOCK((rtable *) _tab, tab);
+ int limit = 2000;
struct rt_import_hook *ih;
node *n, *x;
if (limit <= 0)
{
- birdloop_flag(tab->loop, RTF_CLEANUP);
+ ev_send_loop(tab->loop, tab->prune_event);
return;
}
}
}
- rt_trace(tab, D_EVENTS, "Prune done, scheduling export timer");
- rt_kick_export_settle(tab);
+ rt_trace(tab, D_EVENTS, "Prune done");
+ lfjour_announce_now(&tab->journal);
/* state change 2->0, 3->1 */
if (tab->prune_state &= 1)
- birdloop_flag(tab->loop, RTF_CLEANUP);
+ ev_send_loop(tab->loop, tab->prune_event);
if (tab->trie_new)
{
if (ih->import_state == TIS_FLUSHING)
{
DBG("flushing %s %s rr %u", ih->req->name, tab->name, tab->rr_counter);
- ih->flush_seq = tab->exporter.next_seq;
+ ih->flush_seq = tab->journal.next_seq;
rt_set_import_state(ih, TIS_WAITING);
tab->rr_counter--;
tab->wait_counter++;
+ lfjour_schedule_cleanup(&tab->journal);
}
else if (ih->stale_pruning != ih->stale_pruned)
{
ih->stale_pruned = ih->stale_pruning;
rt_refresh_trace(tab, ih, "table prune after refresh end");
}
-
- /* In some cases, we may want to directly proceed to export cleanup */
- if (tab->wait_counter && (EMPTY_LIST(tab->exporter.hooks) || !tab->exporter.first))
- rt_export_cleanup(tab);
-}
-
-static void
-rt_export_cleanup(struct rtable_private *tab)
-{
- tab->export_used = 0;
-
- u64 min_seq = ~((u64) 0);
- struct rt_pending_export *last_export_to_free = NULL;
- struct rt_pending_export *first = tab->exporter.first;
- int want_prune = 0;
-
- struct rt_export_hook *eh;
- node *n;
- WALK_LIST2(eh, n, tab->exporter.hooks, n)
- {
- switch (atomic_load_explicit(&eh->export_state, memory_order_acquire))
- {
- /* Export cleanup while feeding isn't implemented */
- case TES_FEEDING:
- goto done;
-
- /* States not interfering with export cleanup */
- case TES_DOWN: /* This should not happen at all */
- log(L_WARN "%s: Export cleanup found hook %s in explicit state TES_DOWN", tab->name, eh->req->name);
- /* fall through */
- case TES_HUNGRY: /* Feeding waiting for uncork */
- case TES_STOP: /* No more export will happen on this hook */
- continue;
-
- /* Regular export */
- case TES_READY:
- {
- struct rt_pending_export *last = atomic_load_explicit(&eh->last_export, memory_order_acquire);
- if (!last)
- /* No last export means that the channel has exported nothing since last cleanup */
- goto done;
-
- else if (min_seq > last->seq)
- {
- min_seq = last->seq;
- last_export_to_free = last;
- }
- continue;
- }
-
- default:
- bug("%s: Strange export state of hook %s: %d", tab->name, eh->req->name, atomic_load_explicit(&eh->export_state, memory_order_relaxed));
- }
- }
-
- tab->exporter.first = last_export_to_free ? rt_next_export_fast(last_export_to_free) : NULL;
-
- rt_trace(tab, D_STATES, "Export cleanup, old exporter.first seq %lu, new %lu, min_seq %ld",
- first ? first->seq : 0,
- tab->exporter.first ? tab->exporter.first->seq : 0,
- min_seq);
-
- WALK_LIST2(eh, n, tab->exporter.hooks, n)
- {
- if (atomic_load_explicit(&eh->export_state, memory_order_acquire) != TES_READY)
- continue;
-
- struct rt_pending_export *last = atomic_load_explicit(&eh->last_export, memory_order_acquire);
- if (last == last_export_to_free)
- {
- /* This may fail when the channel managed to export more inbetween. This is OK. */
- atomic_compare_exchange_strong_explicit(
- &eh->last_export, &last, NULL,
- memory_order_release,
- memory_order_relaxed);
-
- DBG("store hook=%p last_export=NULL\n", eh);
- }
- }
-
- while (first && (first->seq <= min_seq))
- {
- ASSERT_DIE(first->new || first->old);
-
- const net_addr *n = first->new ?
- first->new->net :
- first->old->net;
- struct netindex *i = NET_TO_INDEX(n);
- ASSERT_DIE(i->index < tab->routes_block_size);
- net *net = &tab->routes[i->index];
-
- ASSERT_DIE(net->first == first);
-
- if (first == net->last)
- /* The only export here */
- net->last = net->first = NULL;
- else
- /* First is now the next one */
- net->first = atomic_load_explicit(&first->next, memory_order_relaxed);
-
- want_prune += !net->routes && !net->first;
-
- /* For now, the old route may be finally freed */
- if (first->old)
- {
- rt_rte_trace_in(D_ROUTES, first->old->sender->req, first->old, "freed");
- hmap_clear(&tab->id_map, first->old->id);
- rte_free(SKIP_BACK(struct rte_storage, rte, first->old), tab);
- }
-
-#ifdef LOCAL_DEBUG
- memset(first, 0xbd, sizeof(struct rt_pending_export));
-#endif
-
- struct rt_export_block *reb = HEAD(tab->exporter.pending);
- ASSERT_DIE(reb == PAGE_HEAD(first));
-
- u32 pos = (first - &reb->export[0]);
- u32 end = atomic_load_explicit(&reb->end, memory_order_relaxed);
- ASSERT_DIE(pos < end);
-
- struct rt_pending_export *next = NULL;
-
- if (++pos < end)
- next = &reb->export[pos];
- else
- {
- rem_node(&reb->n);
-
-#ifdef LOCAL_DEBUG
- memset(reb, 0xbe, page_size);
-#endif
-
- free_page(reb);
-
- if (EMPTY_LIST(tab->exporter.pending))
- {
- rt_trace(tab, D_EVENTS, "Resetting export seq");
-
- node *n;
- WALK_LIST2(eh, n, tab->exporter.hooks, n)
- {
- if (atomic_load_explicit(&eh->export_state, memory_order_acquire) != TES_READY)
- continue;
-
- ASSERT_DIE(atomic_load_explicit(&eh->last_export, memory_order_acquire) == NULL);
- bmap_reset(&eh->seq_map, 16);
- }
-
- tab->exporter.next_seq = 1;
- }
- else
- {
- reb = HEAD(tab->exporter.pending);
- next = &reb->export[0];
- }
- }
-
- first = next;
- }
-
- rt_check_cork_low(tab);
-
-done:;
- struct rt_import_hook *ih; node *x;
- if (tab->wait_counter)
- WALK_LIST2_DELSAFE(ih, n, x, tab->imports, n)
- if (ih->import_state == TIS_WAITING)
- if (!first || (first->seq >= ih->flush_seq))
- {
- ih->import_state = TIS_CLEARED;
- tab->wait_counter--;
- ev_send(ih->req->list, &ih->announce_event);
- }
-
- if ((tab->gc_counter += want_prune) >= tab->config->gc_threshold)
- rt_kick_prune_timer(tab);
-
- if (tab->export_used)
- birdloop_flag(tab->loop, RTF_CLEANUP);
-
- if (EMPTY_LIST(tab->exporter.pending))
- settle_cancel(&tab->export_settle);
}
static void
if (rt_cork_check(tab->nhu_uncork_event))
{
rt_trace(tab, D_STATES, "Next hop updater corked");
- if ((tab->nhu_state & NHU_RUNNING)
- && !EMPTY_LIST(tab->exporter.pending))
- rt_kick_export_settle(tab);
+
+ if (tab->nhu_state & NHU_RUNNING)
+ lfjour_announce_now(&tab->journal);
tab->nhu_corked = tab->nhu_state;
tab->nhu_state = 0;
/* Finished NHU, cleanup */
rt_trace(tab, D_EVENTS, "NHU done, scheduling export timer");
- rt_kick_export_settle(tab);
/* State change:
* NHU_DIRTY -> NHU_SCHEDULED
if (!tab->cork_active)
return;
- if (tab->deleted || !tab->exporter.first || (tab->exporter.first->seq + tab->cork_threshold.low > tab->exporter.next_seq))
+ if (tab->deleted || (lfjour_pending_items(&tab->journal) < tab->cork_threshold.low))
{
tab->cork_active = 0;
rt_cork_release();
static void
rt_check_cork_high(struct rtable_private *tab)
{
- if (!tab->deleted && !tab->cork_active && tab->exporter.first && (tab->exporter.first->seq + tab->cork_threshold.high <= tab->exporter.next_seq))
+ if (!tab->deleted && !tab->cork_active && (lfjour_pending_items(&tab->journal) >= tab->cork_threshold.high))
{
tab->cork_active = 1;
rt_cork_acquire();
- rt_export_used(&tab->exporter, tab->name, "corked");
+ lfjour_schedule_cleanup(&tab->journal);
+// rt_export_used(&tab->exporter, tab->name, "corked");
rt_trace(tab, D_STATES, "Corked");
}
if (tab->hostcache)
tab->hostcache->req.trace_routes = new->debug;
- struct rt_export_hook *hook; node *n;
- WALK_LIST2(hook, n, tab->exporter.hooks, n)
- if (hook->req->export_one == rt_flowspec_export_one)
- hook->req->trace_routes = new->debug;
+ WALK_TLIST(rt_flowspec_link, ln, &tab->flowspec_links)
+ ln->req.trace_routes = new->debug;
tab->cork_threshold = new->cork_threshold;
static void
rt_feed_done(struct rt_export_hook *c)
{
- c->event.hook = rt_export_hook;
+ c->event->hook = rt_export_hook;
rt_set_export_state(c, BIT32_ALL(TES_FEEDING), TES_READY);
rte_feed_obtain(n, &b->feed[b->cnt], cnt);
+ struct rt_pending_export *first = n->first;
+ struct lfjour_item *last_seen_item = atomic_load_explicit(&c->recipient.last, memory_order_relaxed);
+ struct rt_pending_export *last_seen = last_seen_item ? SKIP_BACK(struct rt_pending_export, li, last_seen_item) : NULL;
+
+ while (last_seen && first && (first->seq <= last_seen->seq))
+ first = first->next;
+
b->aux[b->pos++] = (struct rt_feed_block_aux) {
.start = b->cnt,
- .first = n->first,
- .last = n->last,
+ .first = first,
+ .last = first ? n->last : NULL,
};
b->cnt += cnt;
ASSERT(atomic_load_explicit(&c->export_state, memory_order_relaxed) == TES_FEEDING);
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, exporter, c->table)), tab)
+ RT_LOCKED(c->tab, tab)
{
for (; c->feed_index < tab->routes_block_size; c->feed_index++)
struct rt_export_hook *c = data;
rt_feed_block block = {};
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, exporter, c->table)), tab)
+ RT_LOCKED(c->tab, tab)
{
ASSERT_DIE(c->walk_state);
rt_feed_block block = {};
net *n;
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, exporter, c->table)), tab)
+ RT_LOCKED(c->tab, tab)
{
ASSERT_DIE(atomic_load_explicit(&c->export_state, memory_order_relaxed) == TES_FEEDING);
ASSERT_DIE(c->req->prefilter.mode == TE_ADDR_EQUAL);
rt_feed_block block = {};
net *n;
- RT_LOCKED(RT_PUB(SKIP_BACK(struct rtable_private, exporter, c->table)), tab)
+ RT_LOCKED(c->tab, tab)
{
ASSERT_DIE(atomic_load_explicit(&c->export_state, memory_order_relaxed) == TES_FEEDING);
ASSERT_DIE(c->req->prefilter.mode == TE_ADDR_FOR);