]> git.ipfire.org Git - thirdparty/bird.git/commitdiff
Nest: Use bitmaps to keep track of exported routes
authorOndrej Zajicek (work) <santiago@crfreenet.org>
Mon, 9 Sep 2019 00:55:32 +0000 (02:55 +0200)
committerOndrej Zajicek (work) <santiago@crfreenet.org>
Tue, 26 Nov 2019 17:39:25 +0000 (18:39 +0100)
Use a hierarchical bitmap in a routing table to assign ids to routes, and
then use bitmaps (indexed by route id) in channels to keep track whether
routes were exported. This avoids unreliable and inefficient re-evaluation
of filters for old routes in order to determine whether they were exported.

lib/bitmap.c
lib/bitmap.h
nest/config.Y
nest/proto.c
nest/protocol.h
nest/route.h
nest/rt-show.c
nest/rt-table.c
sysdep/unix/krt.c

index 16bb1464a2e4b587d8a72f2318fb20e9002ed764..b6ea5a3821451145b220baabe84267ea8ea4cceb 100644 (file)
@@ -26,6 +26,13 @@ bmap_init(struct bmap *b, pool *p, uint size)
   b->data = mb_allocz(p, b->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)
 {
index df2945a10ac7644f4fb5c3703813ec181151f9ef..0093cd1884e571dc7330ea0156472a8d893da20b 100644 (file)
@@ -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);
 
index 8fe060c522d77e42d1cd0e3929293fda14f90442..bd1157c640c83dd94fe94e8157c430d15191aebd 100644 (file)
@@ -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 */
index 6beca56d87eaefb85a037eac6d5aca34c8133f5a..ac34d86f3659c93f4f06272d2eeafd714ba43418 100644 (file)
@@ -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);
index c664c1e6af40e499fe9bb31e5d0f2d734795edb0..e4ec7fa2f17bb1468217d9700d8d5abb4bc6a40b 100644 (file)
@@ -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_*) */
index cd4b75db883135ff3a8fc686d61e3a67fbe8a952..434b9b341126532f48e6d75dc02d59a80448c2b2 100644 (file)
@@ -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
index 5114e5f0c13f9b7b663d7dade2c575971f470230..0318f4f06e1b7eed8c321f816021a75ac4c1a95e 100644 (file)
@@ -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;
index df7c1634293035b7a20f4b260dcfbc242a889936..4adc278ec787ea5f0f76089b6fc909b7ade4a628 100644 (file)
@@ -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))
index 27868fab61e15deb58513d0706a56c41d64e5b0f..691efb44b74951a8c395b1f2ecb623e1be7d8cc4 100644 (file)
@@ -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: