bgpd: batch peer connection error clearing

When peer connections encounter errors, attempt to batch some
of the clearing processing that occurs. Add a new batch object,
add multiple peers to it, if possible. Do one rib walk for the
batch, rather than one walk per peer. Use a handler callback
per batch to check and remove peers' path-infos, rather than
a work-queue and callback per peer. The original clearing code
remains; it's used for single peers.

Signed-off-by: Mark Stapp <mjs@cisco.com>
This commit is contained in:
Mark Stapp 2024-10-01 16:30:44 -04:00 committed by Donald Sharp
parent 020245befd
commit 58f924d287
8 changed files with 613 additions and 74 deletions

View file

@ -1281,7 +1281,8 @@ void bgp_fsm_change_status(struct peer_connection *connection,
* Clearing * Clearing
* (or Deleted). * (or Deleted).
*/ */
if (!work_queue_is_scheduled(peer->clear_node_queue) && if (!CHECK_FLAG(peer->flags, PEER_FLAG_CLEARING_BATCH) &&
!work_queue_is_scheduled(peer->clear_node_queue) &&
status != Deleted) status != Deleted)
BGP_EVENT_ADD(connection, Clearing_Completed); BGP_EVENT_ADD(connection, Clearing_Completed);
} }

View file

@ -136,4 +136,6 @@ DECLARE_MTYPE(BGP_SOFT_VERSION);
DECLARE_MTYPE(BGP_EVPN_OVERLAY); DECLARE_MTYPE(BGP_EVPN_OVERLAY);
DECLARE_MTYPE(CLEARING_BATCH);
#endif /* _QUAGGA_BGP_MEMORY_H */ #endif /* _QUAGGA_BGP_MEMORY_H */

View file

@ -4149,62 +4149,3 @@ void bgp_send_delayed_eor(struct bgp *bgp)
for (ALL_LIST_ELEMENTS(bgp->peer, node, nnode, peer)) for (ALL_LIST_ELEMENTS(bgp->peer, node, nnode, peer))
bgp_write_proceed_actions(peer); bgp_write_proceed_actions(peer);
} }
/*
* Task callback in the main pthread to handle socket error
* encountered in the io pthread. We avoid having the io pthread try
* to enqueue fsm events or mess with the peer struct.
*/
/* Max number of peers to process without rescheduling */
#define BGP_CONN_ERROR_DEQUEUE_MAX 10
void bgp_packet_process_error(struct event *thread)
{
struct peer_connection *connection;
struct peer *peer;
struct bgp *bgp;
int counter = 0;
bool more_p = false;
bgp = EVENT_ARG(thread);
/* Dequeue peers from the error list */
while ((peer = bgp_dequeue_conn_err_peer(bgp, &more_p)) != NULL) {
connection = peer->connection;
if (bgp_debug_neighbor_events(peer))
zlog_debug("%s [Event] BGP error %d on fd %d",
peer->host, peer->connection_errcode,
connection->fd);
/* Closed connection or error on the socket */
if (peer_established(connection)) {
if ((CHECK_FLAG(peer->flags, PEER_FLAG_GRACEFUL_RESTART)
|| CHECK_FLAG(peer->flags,
PEER_FLAG_GRACEFUL_RESTART_HELPER))
&& CHECK_FLAG(peer->sflags, PEER_STATUS_NSF_MODE)) {
peer->last_reset = PEER_DOWN_NSF_CLOSE_SESSION;
SET_FLAG(peer->sflags, PEER_STATUS_NSF_WAIT);
} else
peer->last_reset = PEER_DOWN_CLOSE_SESSION;
}
/* No need for keepalives, if enabled */
bgp_keepalives_off(connection);
bgp_event_update(connection, peer->connection_errcode);
counter++;
if (counter >= BGP_CONN_ERROR_DEQUEUE_MAX)
break;
}
/* Reschedule event if necessary */
if (more_p)
bgp_conn_err_reschedule(bgp);
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: dequeued and processed %d peers", __func__,
counter);
}

View file

@ -82,8 +82,6 @@ extern void bgp_process_packet(struct event *event);
extern void bgp_send_delayed_eor(struct bgp *bgp); extern void bgp_send_delayed_eor(struct bgp *bgp);
/* Task callback to handle socket error encountered in the io pthread */
void bgp_packet_process_error(struct event *thread);
extern struct bgp_notify extern struct bgp_notify
bgp_notify_decapsulate_hard_reset(struct bgp_notify *notify); bgp_notify_decapsulate_hard_reset(struct bgp_notify *notify);
extern bool bgp_has_graceful_restart_notification(struct peer *peer); extern bool bgp_has_graceful_restart_notification(struct peer *peer);

View file

@ -80,6 +80,8 @@
DEFINE_MTYPE_STATIC(BGPD, BGP_EOIU_MARKER_INFO, "BGP EOIU Marker info"); DEFINE_MTYPE_STATIC(BGPD, BGP_EOIU_MARKER_INFO, "BGP EOIU Marker info");
DEFINE_MTYPE_STATIC(BGPD, BGP_METAQ, "BGP MetaQ"); DEFINE_MTYPE_STATIC(BGPD, BGP_METAQ, "BGP MetaQ");
/* Memory for batched clearing of peers from the RIB */
DEFINE_MTYPE(BGPD, CLEARING_BATCH, "Clearing batch");
DEFINE_HOOK(bgp_snmp_update_stats, DEFINE_HOOK(bgp_snmp_update_stats,
(struct bgp_dest *rn, struct bgp_path_info *pi, bool added), (struct bgp_dest *rn, struct bgp_path_info *pi, bool added),
@ -6454,11 +6456,244 @@ void bgp_clear_route(struct peer *peer, afi_t afi, safi_t safi)
peer_unlock(peer); peer_unlock(peer);
} }
/*
* Callback scheduled to process prefixes/dests for batch clearing; the
* dests were found via a rib walk.
* The one-peer version of this uses a per-peer workqueue to manage
* rescheduling, but we're just using a fixed limit here.
*/
/* Limit the number of dests we'll process per callback */
#define BGP_CLEARING_BATCH_MAX_DESTS 100
static void bgp_clear_batch_dests_task(struct event *event)
{
struct bgp_clearing_info *cinfo = EVENT_ARG(event);
struct bgp_dest *dest;
struct bgp_path_info *pi, *next;
struct bgp_table *table;
struct bgp *bgp;
afi_t afi;
safi_t safi;
int counter = 0;
bgp = cinfo->bgp;
next_dest:
dest = bgp_clearing_batch_next_dest(cinfo);
if (dest == NULL)
goto done;
table = bgp_dest_table(dest);
afi = table->afi;
safi = table->safi;
/* Have to check every path: it is possible that we have multiple paths
* for a prefix from a peer if that peer is using AddPath.
* Note that the clearing action may change the pi list; we try to do
* a "safe" iteration.
*/
for (pi = bgp_dest_get_bgp_path_info(dest); pi; pi = next) {
next = pi ? pi->next : NULL;
if (!bgp_clearing_batch_check_peer(cinfo, pi->peer))
continue;
/* graceful restart STALE flag set. */
if (((CHECK_FLAG(pi->peer->sflags, PEER_STATUS_NSF_WAIT)
&& pi->peer->nsf[afi][safi])
|| CHECK_FLAG(pi->peer->af_sflags[afi][safi],
PEER_STATUS_ENHANCED_REFRESH))
&& !CHECK_FLAG(pi->flags, BGP_PATH_STALE)
&& !CHECK_FLAG(pi->flags, BGP_PATH_UNUSEABLE))
bgp_path_info_set_flag(dest, pi, BGP_PATH_STALE);
else {
/* If this is an EVPN route, process for
* un-import. */
if (safi == SAFI_EVPN)
bgp_evpn_unimport_route(
bgp, afi, safi,
bgp_dest_get_prefix(dest), pi);
/* Handle withdraw for VRF route-leaking and L3VPN */
if (SAFI_UNICAST == safi
&& (bgp->inst_type == BGP_INSTANCE_TYPE_VRF ||
bgp->inst_type == BGP_INSTANCE_TYPE_DEFAULT)) {
vpn_leak_from_vrf_withdraw(bgp_get_default(),
bgp, pi);
}
if (SAFI_MPLS_VPN == safi &&
bgp->inst_type == BGP_INSTANCE_TYPE_DEFAULT) {
vpn_leak_to_vrf_withdraw(pi);
}
bgp_rib_remove(dest, pi, pi->peer, afi, safi);
}
}
/* Unref this dest and table */
bgp_dest_unlock_node(dest);
bgp_table_unlock(table);
counter++;
if (counter < BGP_CLEARING_BATCH_MAX_DESTS)
goto next_dest;
done:
/* If there are still dests to process, reschedule. */
if (bgp_clearing_batch_dests_present(cinfo)) {
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: Batch %p: Rescheduled after processing %d dests",
__func__, cinfo, counter);
event_add_event(bm->master, bgp_clear_batch_dests_task, cinfo,
0, &cinfo->t_sched);
} else {
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: Batch %p: Done after processing %d dests",
__func__, cinfo, counter);
bgp_clearing_batch_completed(cinfo);
}
return;
}
/*
* Walk a single table for batch peer clearing processing
*/
static void clear_batch_table_helper(struct bgp_clearing_info *cinfo,
struct bgp_table *table)
{
struct bgp_dest *dest;
bool force = (cinfo->bgp->process_queue == NULL);
uint32_t examined = 0, queued = 0;
for (dest = bgp_table_top(table); dest; dest = bgp_route_next(dest)) {
struct bgp_path_info *pi, *next;
struct bgp_adj_in *ain;
struct bgp_adj_in *ain_next;
examined++;
ain = dest->adj_in;
while (ain) {
ain_next = ain->next;
if (bgp_clearing_batch_check_peer(cinfo, ain->peer))
bgp_adj_in_remove(&dest, ain);
ain = ain_next;
assert(dest != NULL);
}
for (pi = bgp_dest_get_bgp_path_info(dest); pi; pi = next) {
next = pi->next;
if (!bgp_clearing_batch_check_peer(cinfo, pi->peer))
continue;
queued++;
if (force) {
bgp_path_info_reap(dest, pi);
} else {
/* Unlocked after processing */
bgp_table_lock(bgp_dest_table(dest));
bgp_dest_lock_node(dest);
bgp_clearing_batch_add_dest(cinfo, dest);
break;
}
}
}
if (examined > 0) {
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: %s/%s: examined %u, queued %u",
__func__, afi2str(table->afi),
safi2str(table->safi), examined, queued);
}
}
/*
* RIB-walking helper for batch clearing work: walk all tables, identify
* dests that are affected by the peers in the batch, enqueue the dests for
* async processing.
*/
static void clear_batch_rib_helper(struct bgp_clearing_info *cinfo)
{
afi_t afi;
safi_t safi;
struct bgp_dest *dest;
struct bgp_table *table;
FOREACH_AFI_SAFI (afi, safi) {
/* Identify table to be examined */
if (safi != SAFI_MPLS_VPN && safi != SAFI_ENCAP &&
safi != SAFI_EVPN) {
table = cinfo->bgp->rib[afi][safi];
if (!table)
continue;
clear_batch_table_helper(cinfo, table);
} else {
for (dest = bgp_table_top(cinfo->bgp->rib[afi][safi]);
dest; dest = bgp_route_next(dest)) {
table = bgp_dest_get_bgp_table_info(dest);
if (!table)
continue;
/* TODO -- record the tables we've seen
* and don't repeat any?
*/
clear_batch_table_helper(cinfo, table);
}
}
}
}
/*
* Identify prefixes that need to be cleared for a batch of peers in 'cinfo'.
* The actual clearing processing will be done async...
*/
void bgp_clear_route_batch(struct bgp_clearing_info *cinfo)
{
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: BGP %s, batch %p", __func__,
cinfo->bgp->name_pretty, cinfo);
/* Walk the rib, checking the peers in the batch */
clear_batch_rib_helper(cinfo);
/* If we found some prefixes, schedule a task to begin work. */
if (bgp_clearing_batch_dests_present(cinfo))
event_add_event(bm->master, bgp_clear_batch_dests_task, cinfo,
0, &cinfo->t_sched);
/* NB -- it's the caller's job to clean up, release refs, etc. if
* we didn't find any dests
*/
}
void bgp_clear_route_all(struct peer *peer) void bgp_clear_route_all(struct peer *peer)
{ {
afi_t afi; afi_t afi;
safi_t safi; safi_t safi;
/* We may be able to batch multiple peers' clearing work: check
* and see.
*/
if (bgp_clearing_batch_add_peer(peer->bgp, peer)) {
if (bgp_debug_neighbor_events(peer))
zlog_debug("%s: peer %pBP batched", __func__, peer);
return;
}
if (bgp_debug_neighbor_events(peer))
zlog_debug("%s: peer %pBP", __func__, peer);
FOREACH_AFI_SAFI (afi, safi) FOREACH_AFI_SAFI (afi, safi)
bgp_clear_route(peer, afi, safi); bgp_clear_route(peer, afi, safi);

View file

@ -779,6 +779,9 @@ extern void bgp_soft_reconfig_table_task_cancel(const struct bgp *bgp,
extern bool bgp_soft_reconfig_in(struct peer *peer, afi_t afi, safi_t safi); extern bool bgp_soft_reconfig_in(struct peer *peer, afi_t afi, safi_t safi);
extern void bgp_clear_route(struct peer *, afi_t, safi_t); extern void bgp_clear_route(struct peer *, afi_t, safi_t);
extern void bgp_clear_route_all(struct peer *); extern void bgp_clear_route_all(struct peer *);
/* Clear routes for a batch of peers */
void bgp_clear_route_batch(struct bgp_clearing_info *cinfo);
extern void bgp_clear_adj_in(struct peer *, afi_t, safi_t); extern void bgp_clear_adj_in(struct peer *, afi_t, safi_t);
extern void bgp_clear_stale_route(struct peer *, afi_t, safi_t); extern void bgp_clear_stale_route(struct peer *, afi_t, safi_t);
extern void bgp_set_stale_route(struct peer *peer, afi_t afi, safi_t safi); extern void bgp_set_stale_route(struct peer *peer, afi_t afi, safi_t safi);

View file

@ -89,7 +89,20 @@ DEFINE_HOOK(bgp_instance_state, (struct bgp *bgp), (bgp));
DEFINE_HOOK(bgp_routerid_update, (struct bgp *bgp, bool withdraw), (bgp, withdraw)); DEFINE_HOOK(bgp_routerid_update, (struct bgp *bgp, bool withdraw), (bgp, withdraw));
/* Peers with connection error/failure, per bgp instance */ /* Peers with connection error/failure, per bgp instance */
DECLARE_LIST(bgp_peer_conn_errlist, struct peer, conn_err_link); DECLARE_DLIST(bgp_peer_conn_errlist, struct peer, conn_err_link);
/* List of info about peers that are being cleared from BGP RIBs in a batch */
DECLARE_DLIST(bgp_clearing_info, struct bgp_clearing_info, link);
/* List of dests that need to be processed in a clearing batch */
DECLARE_LIST(bgp_clearing_destlist, struct bgp_clearing_dest, link);
/* Hash of peers in clearing info object */
static int peer_clearing_hash_cmp(const struct peer *p1, const struct peer *p2);
static uint32_t peer_clearing_hashfn(const struct peer *p1);
DECLARE_HASH(bgp_clearing_hash, struct peer, clear_hash_link,
peer_clearing_hash_cmp, peer_clearing_hashfn);
/* BGP process wide configuration. */ /* BGP process wide configuration. */
static struct bgp_master bgp_master; static struct bgp_master bgp_master;
@ -3653,6 +3666,7 @@ peer_init:
/* Init peer connection error info */ /* Init peer connection error info */
pthread_mutex_init(&bgp->peer_errs_mtx, NULL); pthread_mutex_init(&bgp->peer_errs_mtx, NULL);
bgp_peer_conn_errlist_init(&bgp->peer_conn_errlist); bgp_peer_conn_errlist_init(&bgp->peer_conn_errlist);
bgp_clearing_info_init(&bgp->clearing_list);
return bgp; return bgp;
} }
@ -4036,11 +4050,12 @@ int bgp_delete(struct bgp *bgp)
struct bgp *bgp_to_proc = NULL; struct bgp *bgp_to_proc = NULL;
struct bgp *bgp_to_proc_next = NULL; struct bgp *bgp_to_proc_next = NULL;
struct bgp *bgp_default = bgp_get_default(); struct bgp *bgp_default = bgp_get_default();
struct bgp_clearing_info *cinfo;
assert(bgp); assert(bgp);
/* /*
* Iterate the pending dest list and remove all the dest pertaininig to * Iterate the pending dest list and remove all the dest pertaining to
* the bgp under delete. * the bgp under delete.
*/ */
b_ann_cnt = zebra_announce_count(&bm->zebra_announce_head); b_ann_cnt = zebra_announce_count(&bm->zebra_announce_head);
@ -4086,6 +4101,10 @@ int bgp_delete(struct bgp *bgp)
a_l3_cnt); a_l3_cnt);
} }
/* Cleanup for peer connection batching */
while ((cinfo = bgp_clearing_info_first(&bgp->clearing_list)) != NULL)
bgp_clearing_batch_completed(cinfo);
bgp_soft_reconfig_table_task_cancel(bgp, NULL, NULL); bgp_soft_reconfig_table_task_cancel(bgp, NULL, NULL);
/* make sure we withdraw any exported routes */ /* make sure we withdraw any exported routes */
@ -4241,6 +4260,10 @@ int bgp_delete(struct bgp *bgp)
/* Cancel peer connection errors event */ /* Cancel peer connection errors event */
EVENT_OFF(bgp->t_conn_errors); EVENT_OFF(bgp->t_conn_errors);
/* Cleanup for peer connection batching */
while ((cinfo = bgp_clearing_info_pop(&bgp->clearing_list)) != NULL)
bgp_clearing_batch_completed(cinfo);
/* TODO - Other memory may need to be freed - e.g., NHT */ /* TODO - Other memory may need to be freed - e.g., NHT */
#ifdef ENABLE_BGP_VNC #ifdef ENABLE_BGP_VNC
@ -9012,8 +9035,303 @@ void bgp_gr_apply_running_config(void)
} }
} }
/* Hash of peers in clearing info object */
static int peer_clearing_hash_cmp(const struct peer *p1, const struct peer *p2)
{
if (p1 == p2)
return 0;
else if (p1 < p2)
return -1;
else
return 1;
}
static uint32_t peer_clearing_hashfn(const struct peer *p1)
{
return (uint32_t)((intptr_t)p1 & 0xffffffffULL);
}
/* /*
* Enqueue a peer with a connection error to be handled in the main pthread * Free a clearing batch: this really just does the memory cleanup; the
* clearing code is expected to manage the peer, dest, table, etc refcounts
*/
static void bgp_clearing_batch_free(struct bgp *bgp,
struct bgp_clearing_info **pinfo)
{
struct bgp_clearing_info *cinfo = *pinfo;
struct bgp_clearing_dest *destinfo;
if (bgp_clearing_info_anywhere(cinfo))
bgp_clearing_info_del(&bgp->clearing_list, cinfo);
while ((destinfo = bgp_clearing_destlist_pop(&cinfo->destlist)) != NULL)
XFREE(MTYPE_CLEARING_BATCH, destinfo);
bgp_clearing_hash_fini(&cinfo->peers);
XFREE(MTYPE_CLEARING_BATCH, *pinfo);
}
/*
* Done with a peer that was part of a clearing batch
*/
static void bgp_clearing_peer_done(struct peer *peer)
{
UNSET_FLAG(peer->flags, PEER_FLAG_CLEARING_BATCH);
/* Tickle FSM to start moving again */
BGP_EVENT_ADD(peer->connection, Clearing_Completed);
peer_unlock(peer); /* bgp_clear_route */
}
/*
* Initialize a new batch struct for clearing peer(s) from the RIB
*/
static void bgp_clearing_batch_begin(struct bgp *bgp)
{
struct bgp_clearing_info *cinfo;
cinfo = XCALLOC(MTYPE_CLEARING_BATCH, sizeof(struct bgp_clearing_info));
cinfo->bgp = bgp;
/* Init hash of peers and list of dests */
bgp_clearing_hash_init(&cinfo->peers);
bgp_clearing_destlist_init(&cinfo->destlist);
/* Batch is open for more peers */
SET_FLAG(cinfo->flags, BGP_CLEARING_INFO_FLAG_OPEN);
bgp_clearing_info_add_head(&bgp->clearing_list, cinfo);
}
/*
* Close a batch of clearing peers, and begin working on the RIB
*/
static void bgp_clearing_batch_end(struct bgp *bgp)
{
struct bgp_clearing_info *cinfo;
cinfo = bgp_clearing_info_first(&bgp->clearing_list);
assert(cinfo != NULL);
assert(CHECK_FLAG(cinfo->flags, BGP_CLEARING_INFO_FLAG_OPEN));
/* Batch is closed */
UNSET_FLAG(cinfo->flags, BGP_CLEARING_INFO_FLAG_OPEN);
/* If we have no peers to examine, just discard the batch info */
if (bgp_clearing_hash_count(&cinfo->peers) == 0) {
bgp_clearing_batch_free(bgp, &cinfo);
return;
}
/* Do a RIB walk for the current batch. If it finds dests/prefixes
* to work on, this will schedule a task to process
* the dests/prefixes in the batch.
*/
bgp_clear_route_batch(cinfo);
/* If we found no prefixes/dests, just discard the batch,
* remembering that we're holding a ref for each peer.
*/
if (bgp_clearing_destlist_count(&cinfo->destlist) == 0) {
bgp_clearing_batch_completed(cinfo);
}
}
/* Check whether a dest's peer is relevant to a clearing batch */
bool bgp_clearing_batch_check_peer(struct bgp_clearing_info *cinfo,
const struct peer *peer)
{
struct peer *p;
p = bgp_clearing_hash_find(&cinfo->peers, peer);
return (p != NULL);
}
/*
* Check whether a clearing batch has any dests to process
*/
bool bgp_clearing_batch_dests_present(struct bgp_clearing_info *cinfo)
{
return (bgp_clearing_destlist_count(&cinfo->destlist) > 0);
}
/*
* Done with a peer clearing batch; deal with refcounts, free memory
*/
void bgp_clearing_batch_completed(struct bgp_clearing_info *cinfo)
{
struct peer *peer;
struct bgp_dest *dest;
struct bgp_clearing_dest *destinfo;
struct bgp_table *table;
/* Ensure event is not scheduled */
event_cancel_event(bm->master, &cinfo->t_sched);
/* Remove all peers and un-ref */
while ((peer = bgp_clearing_hash_pop(&cinfo->peers)) != NULL)
bgp_clearing_peer_done(peer);
/* Remove any dests/prefixes and unlock */
destinfo = bgp_clearing_destlist_pop(&cinfo->destlist);
while (destinfo) {
dest = destinfo->dest;
XFREE(MTYPE_CLEARING_BATCH, destinfo);
table = bgp_dest_table(dest);
bgp_dest_unlock_node(dest);
bgp_table_unlock(table);
destinfo = bgp_clearing_destlist_pop(&cinfo->destlist);
}
/* Free memory */
bgp_clearing_batch_free(cinfo->bgp, &cinfo);
}
/*
* Add a prefix/dest to a clearing batch
*/
void bgp_clearing_batch_add_dest(struct bgp_clearing_info *cinfo,
struct bgp_dest *dest)
{
struct bgp_clearing_dest *destinfo;
destinfo = XCALLOC(MTYPE_CLEARING_BATCH,
sizeof(struct bgp_clearing_dest));
destinfo->dest = dest;
bgp_clearing_destlist_add_tail(&cinfo->destlist, destinfo);
}
/*
* Return the next dest for batch clear processing
*/
struct bgp_dest *bgp_clearing_batch_next_dest(struct bgp_clearing_info *cinfo)
{
struct bgp_clearing_dest *destinfo;
struct bgp_dest *dest = NULL;
destinfo = bgp_clearing_destlist_pop(&cinfo->destlist);
if (destinfo) {
dest = destinfo->dest;
XFREE(MTYPE_CLEARING_BATCH, destinfo);
}
return dest;
}
/* If a clearing batch is available for 'peer', add it and return 'true',
* else return 'false'.
*/
bool bgp_clearing_batch_add_peer(struct bgp *bgp, struct peer *peer)
{
struct bgp_clearing_info *cinfo;
cinfo = bgp_clearing_info_first(&bgp->clearing_list);
if (cinfo && CHECK_FLAG(cinfo->flags, BGP_CLEARING_INFO_FLAG_OPEN)) {
if (!CHECK_FLAG(peer->flags, PEER_FLAG_CLEARING_BATCH)) {
/* Add a peer ref */
peer_lock(peer);
bgp_clearing_hash_add(&cinfo->peers, peer);
SET_FLAG(peer->flags, PEER_FLAG_CLEARING_BATCH);
}
return true;
}
return false;
}
/*
* Task callback in the main pthread to handle socket errors
* encountered in the io pthread. We avoid having the io pthread try
* to enqueue fsm events or mess with the peer struct.
*/
/* TODO -- should this be configurable? */
/* Max number of peers to process without rescheduling */
#define BGP_CONN_ERROR_DEQUEUE_MAX 10
static void bgp_process_conn_error(struct event *event)
{
struct bgp *bgp;
struct peer *peer;
struct peer_connection *connection;
int counter = 0;
size_t list_count = 0;
bool more_p = false;
bgp = EVENT_ARG(event);
frr_with_mutex (&bgp->peer_errs_mtx) {
peer = bgp_peer_conn_errlist_pop(&bgp->peer_conn_errlist);
list_count =
bgp_peer_conn_errlist_count(&bgp->peer_conn_errlist);
}
/* If we have multiple peers with errors, try to batch some
* clearing work.
*/
if (list_count > 0)
bgp_clearing_batch_begin(bgp);
/* Dequeue peers from the error list */
while (peer != NULL) {
connection = peer->connection;
if (bgp_debug_neighbor_events(peer))
zlog_debug("%s [Event] BGP error %d on fd %d",
peer->host, peer->connection_errcode,
connection->fd);
/* Closed connection or error on the socket */
if (peer_established(connection)) {
if ((CHECK_FLAG(peer->flags, PEER_FLAG_GRACEFUL_RESTART)
|| CHECK_FLAG(peer->flags,
PEER_FLAG_GRACEFUL_RESTART_HELPER))
&& CHECK_FLAG(peer->sflags, PEER_STATUS_NSF_MODE)) {
peer->last_reset = PEER_DOWN_NSF_CLOSE_SESSION;
SET_FLAG(peer->sflags, PEER_STATUS_NSF_WAIT);
} else
peer->last_reset = PEER_DOWN_CLOSE_SESSION;
}
/* No need for keepalives, if enabled */
bgp_keepalives_off(peer->connection);
/* Drive into state-machine changes */
bgp_event_update(connection, peer->connection_errcode);
counter++;
if (counter >= BGP_CONN_ERROR_DEQUEUE_MAX)
break;
peer = bgp_dequeue_conn_err_peer(bgp, &more_p);
}
/* Reschedule event if necessary */
if (more_p)
bgp_conn_err_reschedule(bgp);
/* Done with a clearing batch */
if (list_count > 0)
bgp_clearing_batch_end(bgp);
if (bgp_debug_neighbor_events(NULL))
zlog_debug("%s: dequeued and processed %d peers", __func__,
counter);
}
/*
* Enqueue a peer with a connection error to be handled in the main pthread;
* this is called from the io pthread.
*/ */
int bgp_enqueue_conn_err_peer(struct bgp *bgp, struct peer *peer, int errcode) int bgp_enqueue_conn_err_peer(struct bgp *bgp, struct peer *peer, int errcode)
{ {
@ -9027,7 +9345,7 @@ int bgp_enqueue_conn_err_peer(struct bgp *bgp, struct peer *peer, int errcode)
} }
} }
/* Ensure an event is scheduled */ /* Ensure an event is scheduled */
event_add_event(bm->master, bgp_packet_process_error, bgp, 0, event_add_event(bm->master, bgp_process_conn_error, bgp, 0,
&bgp->t_conn_errors); &bgp->t_conn_errors);
return 0; return 0;
} }
@ -9061,7 +9379,7 @@ struct peer *bgp_dequeue_conn_err_peer(struct bgp *bgp, bool *more_p)
*/ */
void bgp_conn_err_reschedule(struct bgp *bgp) void bgp_conn_err_reschedule(struct bgp *bgp)
{ {
event_add_event(bm->master, bgp_packet_process_error, bgp, 0, event_add_event(bm->master, bgp_process_conn_error, bgp, 0,
&bgp->t_conn_errors); &bgp->t_conn_errors);
} }

View file

@ -386,32 +386,54 @@ struct bgp_mplsvpn_nh_label_bind_cache;
PREDECL_RBTREE_UNIQ(bgp_mplsvpn_nh_label_bind_cache); PREDECL_RBTREE_UNIQ(bgp_mplsvpn_nh_label_bind_cache);
/* List of peers that have connection errors in the io pthread */ /* List of peers that have connection errors in the io pthread */
PREDECL_LIST(bgp_peer_conn_errlist); PREDECL_DLIST(bgp_peer_conn_errlist);
/* List of info about peers that are being cleared from BGP RIBs in a batch */ /* List of info about peers that are being cleared from BGP RIBs in a batch */
PREDECL_LIST(bgp_clearing_info); PREDECL_DLIST(bgp_clearing_info);
/* Hash of peers in clearing info object */ /* Hash of peers in clearing info object */
PREDECL_HASH(bgp_clearing_hash); PREDECL_HASH(bgp_clearing_hash);
/* List of dests that need to be processed in a clearing batch */
PREDECL_LIST(bgp_clearing_destlist);
struct bgp_clearing_dest {
struct bgp_dest *dest;
struct bgp_clearing_destlist_item link;
};
/* Info about a batch of peers that need to be cleared from the RIB. /* Info about a batch of peers that need to be cleared from the RIB.
* If many peers need to be cleared, we process them in batches, taking * If many peers need to be cleared, we process them in batches, taking
* one walk through the RIB for each batch. * one walk through the RIB for each batch. This is only used for "all"
* afi/safis, typically when processing peer connection errors.
*/ */
struct bgp_clearing_info { struct bgp_clearing_info {
/* Owning bgp instance */
struct bgp *bgp;
/* Hash of peers */ /* Hash of peers */
struct bgp_clearing_hash_head peers; struct bgp_clearing_hash_head peers;
/* Flags */
uint32_t flags;
/* List of dests - wrapped by a small wrapper struct */
struct bgp_clearing_destlist_head destlist;
/* Event to schedule/reschedule processing */ /* Event to schedule/reschedule processing */
struct thread *t_sched; struct event *t_sched;
/* RIB dest for rescheduling */ /* TODO -- id, serial number, for debugging/logging? */
struct bgp_dest *last_dest;
/* Linkage for list of batches per-bgp */ /* TODO -- info for rescheduling the RIB walk? future? */
/* Linkage for list of batches per bgp */
struct bgp_clearing_info_item link; struct bgp_clearing_info_item link;
}; };
/* Batch is open, new peers can be added */
#define BGP_CLEARING_INFO_FLAG_OPEN (1 << 0)
/* BGP instance structure. */ /* BGP instance structure. */
struct bgp { struct bgp {
/* AS number of this BGP instance. */ /* AS number of this BGP instance. */
@ -1609,6 +1631,8 @@ struct peer {
#define PEER_FLAG_EXTENDED_LINK_BANDWIDTH (1ULL << 39) #define PEER_FLAG_EXTENDED_LINK_BANDWIDTH (1ULL << 39)
#define PEER_FLAG_DUAL_AS (1ULL << 40) #define PEER_FLAG_DUAL_AS (1ULL << 40)
#define PEER_FLAG_CAPABILITY_LINK_LOCAL (1ULL << 41) #define PEER_FLAG_CAPABILITY_LINK_LOCAL (1ULL << 41)
/* Peer is part of a batch clearing its routes */
#define PEER_FLAG_CLEARING_BATCH (1ULL << 42)
/* /*
*GR-Disabled mode means unset PEER_FLAG_GRACEFUL_RESTART *GR-Disabled mode means unset PEER_FLAG_GRACEFUL_RESTART
@ -2977,6 +3001,23 @@ extern void srv6_function_free(struct bgp_srv6_function *func);
extern void bgp_session_reset_safe(struct peer *peer, struct listnode **nnode); extern void bgp_session_reset_safe(struct peer *peer, struct listnode **nnode);
/* If a clearing batch is available for 'peer', add it and return 'true',
* else return 'false'.
*/
bool bgp_clearing_batch_add_peer(struct bgp *bgp, struct peer *peer);
/* Add a prefix/dest to a clearing batch */
void bgp_clearing_batch_add_dest(struct bgp_clearing_info *cinfo,
struct bgp_dest *dest);
/* Check whether a dest's peer is relevant to a clearing batch */
bool bgp_clearing_batch_check_peer(struct bgp_clearing_info *cinfo,
const struct peer *peer);
/* Check whether a clearing batch has any dests to process */
bool bgp_clearing_batch_dests_present(struct bgp_clearing_info *cinfo);
/* Returns the next dest for batch clear processing */
struct bgp_dest *bgp_clearing_batch_next_dest(struct bgp_clearing_info *cinfo);
/* Done with a peer clearing batch; deal with refcounts, free memory */
void bgp_clearing_batch_completed(struct bgp_clearing_info *cinfo);
#ifdef _FRR_ATTRIBUTE_PRINTFRR #ifdef _FRR_ATTRIBUTE_PRINTFRR
/* clang-format off */ /* clang-format off */
#pragma FRR printfrr_ext "%pBP" (struct peer *) #pragma FRR printfrr_ext "%pBP" (struct peer *)