aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--lib/bitmap.c7
-rw-r--r--lib/bitmap.h1
-rw-r--r--nest/config.Y3
-rw-r--r--nest/proto.c27
-rw-r--r--nest/protocol.h2
-rw-r--r--nest/route.h4
-rw-r--r--nest/rt-show.c12
-rw-r--r--nest/rt-table.c514
-rw-r--r--sysdep/unix/krt.c7
9 files changed, 252 insertions, 325 deletions
diff --git a/lib/bitmap.c b/lib/bitmap.c
index 16bb1464..b6ea5a38 100644
--- a/lib/bitmap.c
+++ b/lib/bitmap.c
@@ -27,6 +27,13 @@ bmap_init(struct bmap *b, pool *p, uint size)
}
void
+bmap_reset(struct bmap *b, uint size)
+{
+ b->size = BIRD_ALIGN(size, 4);
+ memset(b->data, 0, b->size);
+}
+
+void
bmap_grow(struct bmap *b, uint need)
{
uint size = b->size * 2;
diff --git a/lib/bitmap.h b/lib/bitmap.h
index df2945a1..0093cd18 100644
--- a/lib/bitmap.h
+++ b/lib/bitmap.h
@@ -17,6 +17,7 @@ struct bmap
};
void bmap_init(struct bmap *b, pool *p, uint size);
+void bmap_reset(struct bmap *b, uint size);
void bmap_grow(struct bmap *b, uint need);
void bmap_free(struct bmap *b);
diff --git a/nest/config.Y b/nest/config.Y
index 8fe060c5..bd1157c6 100644
--- a/nest/config.Y
+++ b/nest/config.Y
@@ -70,7 +70,7 @@ CF_KEYWORDS(IPV4, IPV6, VPN4, VPN6, ROA4, ROA6, FLOW4, FLOW6, SADR, MPLS)
CF_KEYWORDS(RECEIVE, LIMIT, ACTION, WARN, BLOCK, RESTART, DISABLE, KEEP, FILTERED)
CF_KEYWORDS(PASSWORD, FROM, PASSIVE, TO, ID, EVENTS, PACKETS, PROTOCOLS, INTERFACES)
CF_KEYWORDS(ALGORITHM, KEYED, HMAC, MD5, SHA1, SHA256, SHA384, SHA512)
-CF_KEYWORDS(PRIMARY, STATS, COUNT, BY, FOR, COMMANDS, PREEXPORT, NOEXPORT, GENERATE)
+CF_KEYWORDS(PRIMARY, STATS, COUNT, BY, FOR, COMMANDS, PREEXPORT, NOEXPORT, EXPORTED, GENERATE)
CF_KEYWORDS(BGP, PASSWORDS, DESCRIPTION, SORTED)
CF_KEYWORDS(RELOAD, IN, OUT, MRTDUMP, MESSAGES, RESTRICT, MEMORY, IGP_METRIC, CLASS, DSCP)
CF_KEYWORDS(TIMEFORMAT, ISO, SHORT, LONG, ROUTE, PROTOCOL, BASE, LOG, S, MS, US)
@@ -685,6 +685,7 @@ export_mode:
PREEXPORT { $$ = RSEM_PREEXPORT; }
| EXPORT { $$ = RSEM_EXPORT; }
| NOEXPORT { $$ = RSEM_NOEXPORT; }
+ | EXPORTED { $$ = RSEM_EXPORTED; }
;
/* This is ugly hack */
diff --git a/nest/proto.c b/nest/proto.c
index 6beca56d..ac34d86f 100644
--- a/nest/proto.c
+++ b/nest/proto.c
@@ -255,6 +255,22 @@ channel_feed_loop(void *ptr)
return;
}
+ /* Reset export limit if the feed ended with acceptable number of exported routes */
+ struct channel_limit *l = &c->out_limit;
+ if (c->refeeding &&
+ (l->state == PLS_BLOCKED) &&
+ (c->refeed_count <= l->limit) &&
+ (c->stats.exp_routes <= l->limit))
+ {
+ log(L_INFO "Protocol %s resets route export limit (%u)", c->proto->name, l->limit);
+ channel_reset_limit(&c->out_limit);
+
+ /* Continue in feed - it will process routing table again from beginning */
+ c->refeed_count = 0;
+ ev_schedule(c->feed_event);
+ return;
+ }
+
// DBG("Feeding protocol %s finished\n", p->name);
c->export_state = ES_READY;
// proto_log_state_change(p);
@@ -282,6 +298,7 @@ channel_stop_export(struct channel *c)
c->export_state = ES_DOWN;
c->stats.exp_routes = 0;
+ bmap_reset(&c->export_map, 1024);
}
@@ -360,6 +377,9 @@ channel_do_start(struct channel *c)
c->feed_event = ev_new_init(c->proto->pool, channel_feed_loop, c);
+ bmap_init(&c->export_map, c->proto->pool, 1024);
+ memset(&c->stats, 0, sizeof(struct proto_stats));
+
channel_reset_limit(&c->rx_limit);
channel_reset_limit(&c->in_limit);
channel_reset_limit(&c->out_limit);
@@ -391,6 +411,7 @@ channel_do_down(struct channel *c)
if ((c->stats.imp_routes + c->stats.filt_routes) != 0)
log(L_ERR "%s: Channel %s is down but still has some routes", c->proto->name, c->name);
+ bmap_free(&c->export_map);
memset(&c->stats, 0, sizeof(struct proto_stats));
c->in_table = NULL;
@@ -503,10 +524,8 @@ channel_request_feeding(struct channel *c)
rt_feed_channel_abort(c);
}
- channel_reset_limit(&c->out_limit);
-
- /* Hack: reset exp_routes during refeed, and do not decrease it later */
- c->stats.exp_routes = 0;
+ /* Track number of exported routes during refeed */
+ c->refeed_count = 0;
channel_schedule_feed(c, 0); /* Sets ES_FEEDING */
// proto_log_state_change(c);
diff --git a/nest/protocol.h b/nest/protocol.h
index c664c1e6..e4ec7fa2 100644
--- a/nest/protocol.h
+++ b/nest/protocol.h
@@ -507,6 +507,7 @@ struct channel {
struct rtable *table;
const struct filter *in_filter; /* Input filter */
const struct filter *out_filter; /* Output filter */
+ struct bmap export_map; /* Keeps track which routes passed export filter */
struct channel_limit rx_limit; /* Receive limit (for in_keep_filtered) */
struct channel_limit in_limit; /* Input limit */
struct channel_limit out_limit; /* Output limit */
@@ -514,6 +515,7 @@ struct channel {
struct event *feed_event; /* Event responsible for feeding */
struct fib_iterator feed_fit; /* Routing table iterator used during feeding */
struct proto_stats stats; /* Per-channel protocol statistics */
+ u32 refeed_count; /* Number of routes exported during refeed regardless of out_limit */
u8 net_type; /* Routing table network type (NET_*), 0 for undefined */
u8 ra_mode; /* Mode of received route advertisements (RA_*) */
diff --git a/nest/route.h b/nest/route.h
index cd4b75db..434b9b34 100644
--- a/nest/route.h
+++ b/nest/route.h
@@ -10,6 +10,7 @@
#define _BIRD_ROUTE_H_
#include "lib/lists.h"
+#include "lib/bitmap.h"
#include "lib/resource.h"
#include "lib/net.h"
@@ -152,6 +153,7 @@ typedef struct rtable {
int pipe_busy; /* Pipe loop detection */
int use_count; /* Number of protocols using this table */
u32 rt_count; /* Number of routes in the table */
+ struct hmap id_map;
struct hostcache *hostcache;
struct rtable_config *config; /* Configuration of this table */
struct config *deleted; /* Table doesn't exist in current configuration,
@@ -210,6 +212,7 @@ typedef struct rte {
net *net; /* Network this RTE belongs to */
struct channel *sender; /* Channel used to send the route to the routing table */
struct rta *attrs; /* Attributes of this route */
+ u32 id; /* Table specific route id */
byte flags; /* Flags (REF_...) */
byte pflags; /* Protocol-specific flags */
word pref; /* Route preference */
@@ -369,6 +372,7 @@ struct rt_show_data_rtable * rt_show_add_table(struct rt_show_data *d, rtable *t
#define RSEM_PREEXPORT 1 /* Routes ready for export, before filtering */
#define RSEM_EXPORT 2 /* Routes accepted by export filter */
#define RSEM_NOEXPORT 3 /* Routes rejected by export filter */
+#define RSEM_EXPORTED 4 /* Routes marked in export map */
/*
* Route Attributes
diff --git a/nest/rt-show.c b/nest/rt-show.c
index 5114e5f0..0318f4f0 100644
--- a/nest/rt-show.c
+++ b/nest/rt-show.c
@@ -120,9 +120,17 @@ rt_show_net(struct cli *c, net *n, struct rt_show_data *d)
if (ec && (ec->export_state == ES_DOWN))
goto skip;
- /* Special case for merged export */
- if ((d->export_mode == RSEM_EXPORT) && (ec->ra_mode == RA_MERGED))
+ if (d->export_mode == RSEM_EXPORTED)
+ {
+ if (!bmap_test(&ec->export_map, ee->id))
+ goto skip;
+
+ // if (ec->ra_mode != RA_ANY)
+ // pass = 1;
+ }
+ else if ((d->export_mode == RSEM_EXPORT) && (ec->ra_mode == RA_MERGED))
{
+ /* Special case for merged export */
rte *rt_free;
e = rt_export_merged(ec, n, &rt_free, c->show_pool, 1);
pass = 1;
diff --git a/nest/rt-table.c b/nest/rt-table.c
index df7c1634..4adc278e 100644
--- a/nest/rt-table.c
+++ b/nest/rt-table.c
@@ -282,6 +282,7 @@ rte_get_temp(rta *a)
rte *e = sl_alloc(rte_slab);
e->attrs = a;
+ e->id = 0;
e->flags = 0;
e->pref = 0;
return e;
@@ -637,51 +638,25 @@ do_rt_notify(struct channel *c, net *net, rte *new, rte *old, int refeed)
struct proto *p = c->proto;
struct proto_stats *stats = &c->stats;
- /*
- * First, apply export limit.
- *
- * Export route limits has several problems. Because exp_routes
- * counter is reset before refeed, we don't really know whether
- * limit is breached and whether the update is new or not. Therefore
- * the number of really exported routes may exceed the limit
- * temporarily (routes exported before and new routes in refeed).
- *
- * Minor advantage is that if the limit is decreased and refeed is
- * requested, the number of exported routes really decrease.
- *
- * Second problem is that with export limits, we don't know whether
- * old was really exported (it might be blocked by limit). When a
- * withdraw is exported, we announce it even when the previous
- * update was blocked. This is not a big issue, but the same problem
- * is in updating exp_routes counter. Therefore, to be consistent in
- * increases and decreases of exp_routes, we count exported routes
- * regardless of blocking by limits.
- *
- * Similar problem is in handling updates - when a new route is
- * received and blocking is active, the route would be blocked, but
- * when an update for the route will be received later, the update
- * would be propagated (as old != NULL). Therefore, we have to block
- * also non-new updates (contrary to import blocking).
- */
+ if (refeed && new)
+ c->refeed_count++;
+ /* Apply export limit */
struct channel_limit *l = &c->out_limit;
- if (l->action && new)
- {
- if ((!old || refeed) && (stats->exp_routes >= l->limit))
- channel_notify_limit(c, l, PLD_OUT, stats->exp_routes);
-
- if (l->state == PLS_BLOCKED)
- {
- stats->exp_routes++; /* see note above */
- stats->exp_updates_rejected++;
- rte_trace_out(D_FILTERS, p, new, "rejected [limit]");
- new = NULL;
+ if (l->action && !old && new)
+ {
+ if (stats->exp_routes >= l->limit)
+ channel_notify_limit(c, l, PLD_OUT, stats->exp_routes);
- if (!old)
- return;
- }
+ if (l->state == PLS_BLOCKED)
+ {
+ stats->exp_updates_rejected++;
+ rte_trace_out(D_FILTERS, p, new, "rejected [limit]");
+ return;
}
+ }
+ /* Apply export table */
if (c->out_table && !rte_update_out(c, net->n.addr, new, old, refeed))
return;
@@ -690,236 +665,132 @@ do_rt_notify(struct channel *c, net *net, rte *new, rte *old, int refeed)
else
stats->exp_withdraws_accepted++;
- /* Hack: We do not decrease exp_routes during refeed, we instead
- reset exp_routes at the start of refeed. */
+ if (old)
+ {
+ bmap_clear(&c->export_map, old->id);
+ stats->exp_routes--;
+ }
+
if (new)
+ {
+ bmap_set(&c->export_map, new->id);
stats->exp_routes++;
- if (old && !refeed)
- stats->exp_routes--;
+ }
if (p->debug & D_ROUTES)
- {
- if (new && old)
- rte_trace_out(D_ROUTES, p, new, "replaced");
- else if (new)
- rte_trace_out(D_ROUTES, p, new, "added");
- else if (old)
- rte_trace_out(D_ROUTES, p, old, "removed");
- }
+ {
+ if (new && old)
+ rte_trace_out(D_ROUTES, p, new, "replaced");
+ else if (new)
+ rte_trace_out(D_ROUTES, p, new, "added");
+ else if (old)
+ rte_trace_out(D_ROUTES, p, old, "removed");
+ }
+
p->rt_notify(p, c, net, new, old);
}
static void
-rt_notify_basic(struct channel *c, net *net, rte *new0, rte *old0, int refeed)
+rt_notify_basic(struct channel *c, net *net, rte *new, rte *old, int refeed)
{
- struct proto *p = c->proto;
-
- rte *new = new0;
- rte *old = old0;
+ // struct proto *p = c->proto;
rte *new_free = NULL;
- rte *old_free = NULL;
if (new)
c->stats.exp_updates_received++;
else
c->stats.exp_withdraws_received++;
- /*
- * This is a tricky part - we don't know whether route 'old' was exported to
- * protocol 'p' or was filtered by the export filter. We try to run the export
- * filter to know this to have a correct value in 'old' argument of rte_update
- * (and proper filter value).
- *
- * This is broken because 'configure soft' may change filters but keep routes.
- * Refeed cycle is expected to be called after change of the filters and with
- * old == new, therefore we do not even try to run the filter on an old route.
- * This may lead to 'spurious withdraws' but ensure that there are no 'missing
- * withdraws'.
- *
- * This is not completely safe as there is a window between reconfiguration
- * and the end of refeed - if a newly filtered route disappears during this
- * period, proper withdraw is not sent (because old would be also filtered)
- * and the route is not refeeded (because it disappeared before that).
- * This is handled below as a special case.
- */
-
if (new)
new = export_filter(c, new, &new_free, 0);
- if (old && !refeed)
- old = export_filter(c, old, &old_free, 1);
+ if (old && !bmap_test(&c->export_map, old->id))
+ old = NULL;
if (!new && !old)
- {
- /*
- * As mentioned above, 'old' value may be incorrect in some race conditions.
- * We generally ignore it with two exceptions:
- *
- * First, withdraw to pipe protocol. In that case we rather propagate
- * unfiltered withdraws regardless of export filters to ensure that when a
- * protocol is flushed, its routes are removed from all tables. Possible
- * spurious unfiltered withdraws are not problem here as they are ignored if
- * there is no corresponding route at the other end of the pipe.
- *
- * Second, recent filter change. If old route is older than filter change,
- * then it was previously evaluated by a different filter and we do not know
- * whether it was really propagated. In that case we rather send spurious
- * withdraw than do nothing and possibly cause phantom routes.
- *
- * In both cases wqe directly call rt_notify() hook instead of
- * do_rt_notify() to avoid logging and stat counters.
- */
-
- int pipe_withdraw = 0, filter_change = 0;
-#ifdef CONFIG_PIPE
- pipe_withdraw = (p->proto == &proto_pipe) && !new0;
-#endif
- filter_change = old0 && (old0->lastmod <= c->last_tx_filter_change);
-
- if ((pipe_withdraw || filter_change) && (p != old0->sender->proto))
- {
- c->stats.exp_withdraws_accepted++;
- p->rt_notify(p, c, net, NULL, old0);
- }
-
return;
- }
do_rt_notify(c, net, new, old, refeed);
- /* Discard temporary rte's */
+ /* Discard temporary rte */
if (new_free)
rte_free(new_free);
- if (old_free)
- rte_free(old_free);
}
static void
-rt_notify_accepted(struct channel *c, net *net, rte *new_changed, rte *old_changed, rte *before_old, int feed)
+rt_notify_accepted(struct channel *c, net *net, rte *new_changed, rte *old_changed, int refeed)
{
- struct proto *p = c->proto;
-
- rte *r;
+ // struct proto *p = c->proto;
rte *new_best = NULL;
rte *old_best = NULL;
rte *new_free = NULL;
- rte *old_free = NULL;
+ int new_first = 0;
- /* Used to track whether we met old_changed position. If before_old is NULL
- old_changed was the first and we met it implicitly before current best route. */
- int old_meet = old_changed && !before_old;
-
- /* Note that before_old is either NULL or valid (not rejected) route.
- If old_changed is valid, before_old have to be too. If old changed route
- was not valid, caller must use NULL for both old_changed and before_old. */
+ /*
+ * We assume that there are no changes in net route order except (added)
+ * new_changed and (removed) old_changed. Therefore, the function is not
+ * compatible with deterministic_med (where nontrivial reordering can happen
+ * as a result of a route change) and with recomputation of recursive routes
+ * due to next hop update (where many routes can be changed in one step).
+ *
+ * Note that we need this assumption just for optimizations, we could just
+ * run full new_best recomputation otherwise.
+ *
+ * There are three cases:
+ * feed or old_best is old_changed -> we need to recompute new_best
+ * old_best is before new_changed -> new_best is old_best, ignore
+ * old_best is after new_changed -> try new_changed, otherwise old_best
+ */
- if (new_changed)
+ if (net->routes)
c->stats.exp_updates_received++;
else
c->stats.exp_withdraws_received++;
- /* First, find the new_best route - first accepted by filters */
- for (r=net->routes; rte_is_valid(r); r=r->next)
+ /* Find old_best - either old_changed, or route for net->routes */
+ if (old_changed && bmap_test(&c->export_map, old_changed->id))
+ old_best = old_changed;
+ else
+ {
+ for (rte *r = net->routes; rte_is_valid(r); r = r->next)
{
- if (new_best = export_filter(c, r, &new_free, 0))
+ if (bmap_test(&c->export_map, r->id))
+ {
+ old_best = r;
break;
+ }
- /* Note if we walked around the position of old_changed route */
- if (r == before_old)
- old_meet = 1;
- }
-
- /*
- * Second, handle the feed case. That means we do not care for
- * old_best. It is NULL for feed, and the new_best for refeed.
- * For refeed, there is a hack similar to one in rt_notify_basic()
- * to ensure withdraws in case of changed filters
- */
- if (feed)
- {
- if (feed == 2) /* refeed */
- old_best = new_best ? new_best :
- (rte_is_valid(net->routes) ? net->routes : NULL);
- else
- old_best = NULL;
-
- if (!new_best && !old_best)
- return;
-
- goto found;
- }
-
- /*
- * Now, we find the old_best route. Generally, it is the same as the
- * new_best, unless new_best is the same as new_changed or
- * old_changed is accepted before new_best.
- *
- * There are four cases:
- *
- * - We would find and accept old_changed before new_best, therefore
- * old_changed is old_best. In remaining cases we suppose this
- * is not true.
- *
- * - We found no new_best, therefore there is also no old_best and
- * we ignore this withdraw.
- *
- * - We found new_best different than new_changed, therefore
- * old_best is the same as new_best and we ignore this update.
- *
- * - We found new_best the same as new_changed, therefore it cannot
- * be old_best and we have to continue search for old_best.
- *
- * There is also a hack to ensure consistency in case of changed filters.
- * It does not find the proper old_best, just selects a non-NULL route.
- */
-
- /* Hack for changed filters */
- if (old_changed &&
- (p != old_changed->sender->proto) &&
- (old_changed->lastmod <= c->last_tx_filter_change))
- {
- old_best = old_changed;
- goto found;
+ /* Note if new_changed found before old_best */
+ if (r == new_changed)
+ new_first = 1;
}
+ }
- /* First case */
- if (old_meet)
- if (old_best = export_filter(c, old_changed, &old_free, 1))
- goto found;
-
- /* Second case */
- if (!new_best)
- return;
-
- /* Third case, we use r instead of new_best, because export_filter() could change it */
- if (r != new_changed)
- {
- if (new_free)
- rte_free(new_free);
+ /* Find new_best */
+ if ((new_changed == old_changed) || (old_best == old_changed))
+ {
+ /* Feed or old_best changed -> find first accepted by filters */
+ for (rte *r = net->routes; rte_is_valid(r); r = r->next)
+ if (new_best = export_filter(c, r, &new_free, 0))
+ break;
+ }
+ else
+ {
+ /* Other cases -> either new_changed, or old_best (and nothing changed) */
+ if (new_first && (new_changed = export_filter(c, new_changed, &new_free, 0)))
+ new_best = new_changed;
+ else
return;
- }
-
- /* Fourth case */
- for (r=r->next; rte_is_valid(r); r=r->next)
- {
- if (old_best = export_filter(c, r, &old_free, 1))
- goto found;
-
- if (r == before_old)
- if (old_best = export_filter(c, old_changed, &old_free, 1))
- goto found;
- }
+ }
- /* Implicitly, old_best is NULL and new_best is non-NULL */
+ if (!new_best && !old_best)
+ return;
- found:
- do_rt_notify(c, net, new_best, old_best, (feed == 2));
+ do_rt_notify(c, net, new_best, old_best, refeed);
- /* Discard temporary rte's */
+ /* Discard temporary rte */
if (new_free)
rte_free(new_free);
- if (old_free)
- rte_free(old_free);
}
@@ -984,14 +855,10 @@ rt_export_merged(struct channel *c, net *net, rte **rt_free, linpool *pool, int
static void
rt_notify_merged(struct channel *c, net *net, rte *new_changed, rte *old_changed,
- rte *new_best, rte*old_best, int refeed)
+ rte *new_best, rte *old_best, int refeed)
{
// struct proto *p = c->proto;
-
- rte *new_best_free = NULL;
- rte *old_best_free = NULL;
- rte *new_changed_free = NULL;
- rte *old_changed_free = NULL;
+ rte *new_free = NULL;
/* We assume that all rte arguments are either NULL or rte_is_valid() */
@@ -1000,17 +867,11 @@ rt_notify_merged(struct channel *c, net *net, rte *new_changed, rte *old_changed
return;
/* Check whether the change is relevant to the merged route */
- if ((new_best == old_best) && !refeed)
- {
- new_changed = rte_mergable(new_best, new_changed) ?
- export_filter(c, new_changed, &new_changed_free, 1) : NULL;
-
- old_changed = rte_mergable(old_best, old_changed) ?
- export_filter(c, old_changed, &old_changed_free, 1) : NULL;
-
- if (!new_changed && !old_changed)
- return;
- }
+ if ((new_best == old_best) &&
+ (new_changed != old_changed) &&
+ !rte_mergable(new_best, new_changed) &&
+ !rte_mergable(old_best, old_changed))
+ return;
if (new_best)
c->stats.exp_updates_received++;
@@ -1019,69 +880,71 @@ rt_notify_merged(struct channel *c, net *net, rte *new_changed, rte *old_changed
/* Prepare new merged route */
if (new_best)
- new_best = rt_export_merged(c, net, &new_best_free, rte_update_pool, 0);
+ new_best = rt_export_merged(c, net, &new_free, rte_update_pool, 0);
- /* Prepare old merged route (without proper merged next hops) */
- /* There are some issues with running filter on old route - see rt_notify_basic() */
- if (old_best && !refeed)
- old_best = export_filter(c, old_best, &old_best_free, 1);
+ /* Check old merged route */
+ if (old_best && !bmap_test(&c->export_map, old_best->id))
+ old_best = NULL;
- if (new_best || old_best)
- do_rt_notify(c, net, new_best, old_best, refeed);
+ if (!new_best && !old_best)
+ return;
- /* Discard temporary rte's */
- if (new_best_free)
- rte_free(new_best_free);
- if (old_best_free)
- rte_free(old_best_free);
- if (new_changed_free)
- rte_free(new_changed_free);
- if (old_changed_free)
- rte_free(old_changed_free);
+ do_rt_notify(c, net, new_best, old_best, refeed);
+
+ /* Discard temporary rte */
+ if (new_free)
+ rte_free(new_free);
}
/**
* rte_announce - announce a routing table change
* @tab: table the route has been added to
- * @type: type of route announcement (RA_OPTIMAL or RA_ANY)
+ * @type: type of route announcement (RA_UNDEF or RA_ANY)
* @net: network in question
- * @new: the new route to be announced
- * @old: the previous route for the same network
+ * @new: the new or changed route
+ * @old: the previous route replaced by the new one
* @new_best: the new best route for the same network
* @old_best: the previous best route for the same network
- * @before_old: The previous route before @old for the same network.
- * If @before_old is NULL @old was the first.
*
- * This function gets a routing table update and announces it
- * to all protocols that acccepts given type of route announcement
- * and are connected to the same table by their announcement hooks.
+ * This function gets a routing table update and announces it to all protocols
+ * that are connected to the same table by their channels.
+ *
+ * There are two ways of how routing table changes are announced. First, there
+ * is a change of just one route in @net (which may caused a change of the best
+ * route of the network). In this case @new and @old describes the changed route
+ * and @new_best and @old_best describes best routes. Other routes are not
+ * affected, but in sorted table the order of other routes might change.
*
- * Route announcement of type %RA_OPTIMAL si generated when optimal
- * route (in routing table @tab) changes. In that case @old stores the
- * old optimal route.
+ * Second, There is a bulk change of multiple routes in @net, with shared best
+ * route selection. In such case separate route changes are described using
+ * @type of %RA_ANY, with @new and @old specifying the changed route, while
+ * @new_best and @old_best are NULL. After that, another notification is done
+ * where @new_best and @old_best are filled (may be the same), but @new and @old
+ * are NULL.
*
- * Route announcement of type %RA_ANY si generated when any route (in
- * routing table @tab) changes In that case @old stores the old route
- * from the same protocol.
+ * The function announces the change to all associated channels. For each
+ * channel, an appropriate preprocessing is done according to channel &ra_mode.
+ * For example, %RA_OPTIMAL channels receive just changes of best routes.
*
- * For each appropriate protocol, we first call its preexport()
- * hook which performs basic checks on the route (each protocol has a
- * right to veto or force accept of the route before any filter is
- * asked) and adds default values of attributes specific to the new
- * protocol (metrics, tags etc.). Then it consults the protocol's
- * export filter and if it accepts the route, the rt_notify() hook of
- * the protocol gets called.
+ * In general, we first call preexport() hook of a protocol, which performs
+ * basic checks on the route (each protocol has a right to veto or force accept
+ * of the route before any filter is asked). Then we consult an export filter
+ * of the channel and verify the old route in an export map of the channel.
+ * Finally, the rt_notify() hook of the protocol gets called.
+ *
+ * Note that there are also calls of rt_notify() hooks due to feed, but that is
+ * done outside of scope of rte_announce().
*/
static void
-rte_announce(rtable *tab, unsigned type, net *net, rte *new, rte *old,
- rte *new_best, rte *old_best, rte *before_old)
+rte_announce(rtable *tab, uint type, net *net, rte *new, rte *old,
+ rte *new_best, rte *old_best)
{
if (!rte_is_valid(new))
new = NULL;
if (!rte_is_valid(old))
- old = before_old = NULL;
+ old = NULL;
if (!rte_is_valid(new_best))
new_best = NULL;
@@ -1089,15 +952,15 @@ rte_announce(rtable *tab, unsigned type, net *net, rte *new, rte *old,
if (!rte_is_valid(old_best))
old_best = NULL;
- if (!old && !new)
+ if (!new && !old && !new_best && !old_best)
return;
- if (type == RA_OPTIMAL)
+ if (new_best != old_best)
{
- if (new)
- new->sender->stats.pref_routes++;
- if (old)
- old->sender->stats.pref_routes--;
+ if (new_best)
+ new_best->sender->stats.pref_routes++;
+ if (old_best)
+ old_best->sender->stats.pref_routes--;
if (tab->hostcache)
rt_notify_hostcache(tab, net);
@@ -1105,18 +968,34 @@ rte_announce(rtable *tab, unsigned type, net *net, rte *new, rte *old,
struct channel *c; node *n;
WALK_LIST2(c, n, tab->channels, table_node)
+ {
+ if (c->export_state == ES_DOWN)
+ continue;
+
+ if (type && (type != c->ra_mode))
+ continue;
+
+ switch (c->ra_mode)
{
- if (c->export_state == ES_DOWN)
- continue;
+ case RA_OPTIMAL:
+ if (new_best != old_best)
+ rt_notify_basic(c, net, new_best, old_best, 0);
+ break;
+
+ case RA_ANY:
+ if (new != old)
+ rt_notify_basic(c, net, new, old, 0);
+ break;
+
+ case RA_ACCEPTED:
+ rt_notify_accepted(c, net, new, old, 0);
+ break;
- if (c->ra_mode == type)
- if (type == RA_ACCEPTED)
- rt_notify_accepted(c, net, new, old, before_old, 0);
- else if (type == RA_MERGED)
- rt_notify_merged(c, net, new, old, new_best, old_best, 0);
- else
- rt_notify_basic(c, net, new, old, 0);
+ case RA_MERGED:
+ rt_notify_merged(c, net, new, old, new_best, old_best, 0);
+ break;
}
+ }
}
static inline int
@@ -1418,7 +1297,17 @@ rte_recalculate(struct channel *c, net *net, rte *new, struct rte_src *src)
}
if (new)
- new->lastmod = current_time();
+ {
+ new->lastmod = current_time();
+
+ if (!old)
+ {
+ new->id = hmap_first_zero(&table->id_map);
+ hmap_set(&table->id_map, new->id);
+ }
+ else
+ new->id = old->id;
+ }
/* Log the route change */
if (p->debug & D_ROUTES)
@@ -1437,12 +1326,7 @@ rte_recalculate(struct channel *c, net *net, rte *new, struct rte_src *src)
}
/* Propagate the route change */
- rte_announce(table, RA_ANY, net, new, old, NULL, NULL, NULL);
- if (net->routes != old_best)
- rte_announce(table, RA_OPTIMAL, net, net->routes, old_best, NULL, NULL, NULL);
- if (table->config->sorted)
- rte_announce(table, RA_ACCEPTED, net, new, old, NULL, NULL, before_old);
- rte_announce(table, RA_MERGED, net, new, old, net->routes, old_best, NULL);
+ rte_announce(table, RA_UNDEF, net, new, old, net->routes, old_best);
if (!net->routes &&
(table->gc_counter++ >= table->config->gc_max_ops) &&
@@ -1455,7 +1339,12 @@ rte_recalculate(struct channel *c, net *net, rte *new, struct rte_src *src)
p->rte_insert(net, new);
if (old)
- rte_free_quick(old);
+ {
+ if (!new)
+ hmap_clear(&table->id_map, old->id);
+
+ rte_free_quick(old);
+ }
}
static int rte_update_nest_cnt; /* Nesting counter to allow recursive updates */
@@ -1642,11 +1531,11 @@ rte_update2(struct channel *c, const net_addr *n, rte *new, struct rte_src *src)
/* Independent call to rte_announce(), used from next hop
recalculation, outside of rte_update(). new must be non-NULL */
static inline void
-rte_announce_i(rtable *tab, unsigned type, net *net, rte *new, rte *old,
+rte_announce_i(rtable *tab, uint type, net *net, rte *new, rte *old,
rte *new_best, rte *old_best)
{
rte_update_lock();
- rte_announce(tab, type, net, new, old, new_best, old_best, NULL);
+ rte_announce(tab, type, net, new, old, new_best, old_best);
rte_update_unlock();
}
@@ -1905,6 +1794,9 @@ rt_setup(pool *p, rtable *t, struct rtable_config *cf)
fib_init(&t->fib, p, t->addr_type, sizeof(net), OFFSETOF(net, n), 0, NULL);
init_list(&t->channels);
+ hmap_init(&t->id_map, p, 1024);
+ hmap_set(&t->id_map, 0);
+
t->rt_event = ev_new_init(p, rt_event, t);
t->gc_time = current_time();
}
@@ -2200,8 +2092,8 @@ rt_next_hop_update_net(rtable *tab, net *n)
new = rt_next_hop_update_rte(tab, e);
*k = new;
- rte_announce_i(tab, RA_ANY, n, new, e, NULL, NULL);
rte_trace_in(D_ROUTES, new->sender->proto, new, "updated");
+ rte_announce_i(tab, RA_ANY, n, new, e, NULL, NULL);
/* Call a pre-comparison hook */
/* Not really an efficient way to compute this */
@@ -2239,13 +2131,10 @@ rt_next_hop_update_net(rtable *tab, net *n)
/* Announce the new best route */
if (new != old_best)
- {
- rte_announce_i(tab, RA_OPTIMAL, n, new, old_best, NULL, NULL);
- rte_trace_in(D_ROUTES, new->sender->proto, new, "updated [best]");
- }
+ rte_trace_in(D_ROUTES, new->sender->proto, new, "updated [best]");
- /* FIXME: Better announcement of merged routes */
- rte_announce_i(tab, RA_MERGED, n, new, old_best, new, old_best);
+ /* Propagate changes */
+ rte_announce_i(tab, RA_UNDEF, n, NULL, NULL, n->routes, old_best);
if (free_old_best)
rte_free_quick(old_best);
@@ -2351,6 +2240,7 @@ rt_unlock_table(rtable *r)
rt_free_hostcache(r);
rem_node(&r->n);
fib_free(&r->fib);
+ hmap_free(&r->id_map);
rfree(r->rt_event);
mb_free(r);
config_del_obstacle(conf);
@@ -2428,11 +2318,11 @@ do_feed_channel(struct channel *c, net *n, rte *e)
{
rte_update_lock();
if (c->ra_mode == RA_ACCEPTED)
- rt_notify_accepted(c, n, e, NULL, NULL, c->refeeding ? 2 : 1);
+ rt_notify_accepted(c, n, NULL, NULL, c->refeeding);
else if (c->ra_mode == RA_MERGED)
- rt_notify_merged(c, n, NULL, NULL, e, c->refeeding ? e : NULL, c->refeeding);
+ rt_notify_merged(c, n, NULL, NULL, e, e, c->refeeding);
else /* RA_BASIC */
- rt_notify_basic(c, n, e, c->refeeding ? e : NULL, c->refeeding);
+ rt_notify_basic(c, n, e, e, c->refeeding);
rte_update_unlock();
}
@@ -2468,8 +2358,6 @@ rt_feed_channel(struct channel *c)
return 0;
}
- /* FIXME: perhaps we should change feed for RA_ACCEPTED to not use 'new' */
-
if ((c->ra_mode == RA_OPTIMAL) ||
(c->ra_mode == RA_ACCEPTED) ||
(c->ra_mode == RA_MERGED))
diff --git a/sysdep/unix/krt.c b/sysdep/unix/krt.c
index 27868fab..691efb44 100644
--- a/sysdep/unix/krt.c
+++ b/sysdep/unix/krt.c
@@ -736,11 +736,8 @@ krt_prune(struct krt_proto *p)
switch (verdict)
{
case KRF_CREATE:
- if (new && (n->n.flags & KRF_INSTALLED))
- {
- krt_trace_in(p, new, "reinstalling");
- krt_replace_rte(p, n, new, NULL);
- }
+ krt_trace_in(p, new, "reinstalling");
+ krt_replace_rte(p, n, new, NULL);
break;
case KRF_SEEN:
case KRF_IGNORE: