0
0
mirror of https://gitlab.nic.cz/labs/bird.git synced 2025-01-03 07:31:54 +00:00

Decoupling loops from threads to allow fixed thread count

On large configurations, too many threads would spawn with one thread
per loop. Therefore, threads may now run multiple loops at once. The
thread count is configurable and may be changed during run. All threads
are spawned on startup.

This change helps with memory bloating. BIRD filters need large
temporary memory blocks to store their stack and also memory management
keeps its hot page storage per-thread.

Known bugs:
* Thread autobalancing is not yet implemented.
* Low latency loops are executed together with standard loops.
This commit is contained in:
Maria Matejka 2023-01-19 10:49:47 +01:00
parent 4d8d81f144
commit 84c298465f
17 changed files with 676 additions and 248 deletions

View File

@ -54,6 +54,7 @@ struct config {
char *err_file_name; /* File name containing error */ char *err_file_name; /* File name containing error */
char *file_name; /* Name of main configuration file */ char *file_name; /* Name of main configuration file */
int file_fd; /* File descriptor of main configuration file */ int file_fd; /* File descriptor of main configuration file */
int thread_count; /* How many worker threads to prefork */
struct sym_scope *root_scope; /* Scope for root symbols */ struct sym_scope *root_scope; /* Scope for root symbols */
int obstacle_count; /* Number of items blocking freeing of this config */ int obstacle_count; /* Number of items blocking freeing of this config */

View File

@ -532,6 +532,12 @@ include "tablename.conf";;
killed by abort signal. The timeout has effective granularity of killed by abort signal. The timeout has effective granularity of
seconds, zero means disabled. Default: disabled (0). seconds, zero means disabled. Default: disabled (0).
<tag><label id="opt-threads">threads <m/number/</tag>
Set how many worker threads should BIRD spawn. Tests show that every
thread can utilize one complete CPU core, therefore you probably want to
keep at least one free core. The maximum feasible thread count heavily
depends on the actual workload and must be determined by testing or estimation. Default: 1
<tag><label id="opt-mrtdump">mrtdump "<m/filename/"</tag> <tag><label id="opt-mrtdump">mrtdump "<m/filename/"</tag>
Set MRTdump file name. This option must be specified to allow MRTdump Set MRTdump file name. This option must be specified to allow MRTdump
feature. Default: no dump file. feature. Default: no dump file.

View File

@ -61,6 +61,7 @@ Reply codes of BIRD command-line interface
1023 Show Babel interfaces 1023 Show Babel interfaces
1024 Show Babel neighbors 1024 Show Babel neighbors
1025 Show Babel entries 1025 Show Babel entries
1026 Show threads
8000 Reply too long 8000 Reply too long
8001 Route not found 8001 Route not found

View File

@ -261,7 +261,7 @@ ev_send(event_list *l, event *e)
memory_order_acq_rel, memory_order_acquire)); memory_order_acq_rel, memory_order_acquire));
edlog(l, e, next, 4, EDL_SEND); edlog(l, e, next, 4, EDL_SEND);
birdloop_ping(l->loop); if (l->loop) birdloop_ping(l->loop);
} }
void io_log_event(void *hook, void *data); void io_log_event(void *hook, void *data);

View File

@ -11,7 +11,6 @@
#include "lib/resource.h" #include "lib/resource.h"
#include "lib/locking.h" #include "lib/locking.h"
#include "lib/rcu.h"
#include <stdatomic.h> #include <stdatomic.h>

View File

@ -14,6 +14,7 @@ struct domain_generic;
/* Here define the global lock order; first to last. */ /* Here define the global lock order; first to last. */
struct lock_order { struct lock_order {
struct domain_generic *the_bird; struct domain_generic *the_bird;
struct domain_generic *control;
struct domain_generic *proto; struct domain_generic *proto;
struct domain_generic *service; struct domain_generic *service;
struct domain_generic *rtable; struct domain_generic *rtable;
@ -34,6 +35,9 @@ struct domain_generic *domain_new(const char *name, uint order);
#define DOMAIN_FREE(type, d) domain_free((d).type) #define DOMAIN_FREE(type, d) domain_free((d).type)
void domain_free(struct domain_generic *); void domain_free(struct domain_generic *);
#define DOMAIN_NAME(type, d) domain_name((d).type)
const char *domain_name(struct domain_generic *);
#define DOMAIN_NULL(type) (DOMAIN(type)) {} #define DOMAIN_NULL(type) (DOMAIN(type)) {}
#define LOCK_DOMAIN(type, d) do_lock(((d).type), &(locking_stack.type)) #define LOCK_DOMAIN(type, d) do_lock(((d).type), &(locking_stack.type))

View File

@ -17,11 +17,11 @@
#include "lib/locking.h" #include "lib/locking.h"
_Atomic uint rcu_gp_ctl = RCU_NEST_CNT; _Atomic uint rcu_gp_ctl = RCU_NEST_CNT;
_Thread_local struct rcu_birdloop *this_rcu_birdloop = NULL; _Thread_local struct rcu_thread *this_rcu_thread = NULL;
static list rcu_birdloop_list; static list rcu_thread_list;
static struct rcu_birdloop main_rcu_birdloop; static struct rcu_thread main_rcu_thread;
DEFINE_DOMAIN(resource); DEFINE_DOMAIN(resource);
static DOMAIN(resource) rcu_domain; static DOMAIN(resource) rcu_domain;
@ -37,8 +37,8 @@ static void
update_counter_and_wait(void) update_counter_and_wait(void)
{ {
atomic_fetch_xor(&rcu_gp_ctl, RCU_GP_PHASE); atomic_fetch_xor(&rcu_gp_ctl, RCU_GP_PHASE);
struct rcu_birdloop *rc; struct rcu_thread *rc;
WALK_LIST(rc, rcu_birdloop_list) WALK_LIST(rc, rcu_thread_list)
while (rcu_gp_ongoing(&rc->ctl)) while (rcu_gp_ongoing(&rc->ctl))
birdloop_yield(); birdloop_yield();
} }
@ -53,19 +53,19 @@ synchronize_rcu(void)
} }
void void
rcu_birdloop_start(struct rcu_birdloop *rc) rcu_thread_start(struct rcu_thread *rc)
{ {
LOCK_DOMAIN(resource, rcu_domain); LOCK_DOMAIN(resource, rcu_domain);
add_tail(&rcu_birdloop_list, &rc->n); add_tail(&rcu_thread_list, &rc->n);
this_rcu_birdloop = rc; this_rcu_thread = rc;
UNLOCK_DOMAIN(resource, rcu_domain); UNLOCK_DOMAIN(resource, rcu_domain);
} }
void void
rcu_birdloop_stop(struct rcu_birdloop *rc) rcu_thread_stop(struct rcu_thread *rc)
{ {
LOCK_DOMAIN(resource, rcu_domain); LOCK_DOMAIN(resource, rcu_domain);
this_rcu_birdloop = NULL; this_rcu_thread = NULL;
rem_node(&rc->n); rem_node(&rc->n);
UNLOCK_DOMAIN(resource, rcu_domain); UNLOCK_DOMAIN(resource, rcu_domain);
} }
@ -74,6 +74,6 @@ void
rcu_init(void) rcu_init(void)
{ {
rcu_domain = DOMAIN_NEW(resource, "Read-Copy-Update"); rcu_domain = DOMAIN_NEW(resource, "Read-Copy-Update");
init_list(&rcu_birdloop_list); init_list(&rcu_thread_list);
rcu_birdloop_start(&main_rcu_birdloop); rcu_thread_start(&main_rcu_thread);
} }

View File

@ -21,33 +21,33 @@
extern _Atomic uint rcu_gp_ctl; extern _Atomic uint rcu_gp_ctl;
struct rcu_birdloop { struct rcu_thread {
node n; node n;
_Atomic uint ctl; _Atomic uint ctl;
}; };
extern _Thread_local struct rcu_birdloop *this_rcu_birdloop; extern _Thread_local struct rcu_thread *this_rcu_thread;
static inline void rcu_read_lock(void) static inline void rcu_read_lock(void)
{ {
uint cmp = atomic_load_explicit(&this_rcu_birdloop->ctl, memory_order_acquire); uint cmp = atomic_load_explicit(&this_rcu_thread->ctl, memory_order_acquire);
if (cmp & RCU_NEST_MASK) if (cmp & RCU_NEST_MASK)
atomic_store_explicit(&this_rcu_birdloop->ctl, cmp + RCU_NEST_CNT, memory_order_relaxed); atomic_store_explicit(&this_rcu_thread->ctl, cmp + RCU_NEST_CNT, memory_order_relaxed);
else else
atomic_store(&this_rcu_birdloop->ctl, atomic_load_explicit(&rcu_gp_ctl, memory_order_acquire)); atomic_store(&this_rcu_thread->ctl, atomic_load_explicit(&rcu_gp_ctl, memory_order_acquire));
} }
static inline void rcu_read_unlock(void) static inline void rcu_read_unlock(void)
{ {
atomic_fetch_sub(&this_rcu_birdloop->ctl, RCU_NEST_CNT); atomic_fetch_sub(&this_rcu_thread->ctl, RCU_NEST_CNT);
} }
void synchronize_rcu(void); void synchronize_rcu(void);
/* Registering and unregistering a birdloop. To be called from birdloop implementation */ /* Registering and unregistering a birdloop. To be called from birdloop implementation */
void rcu_birdloop_start(struct rcu_birdloop *); void rcu_thread_start(struct rcu_thread *);
void rcu_birdloop_stop(struct rcu_birdloop *); void rcu_thread_stop(struct rcu_thread *);
/* Run this from resource init */ /* Run this from resource init */
void rcu_init(void); void rcu_init(void);

View File

@ -101,6 +101,12 @@ mrtdump_base:
; ;
conf: THREADS expr {
if ($2 < 1) cf_error("Number of threads must be at least one.");
new_config->thread_count = $2;
}
conf: debug_unix ; conf: debug_unix ;
debug_unix: debug_unix:
@ -145,6 +151,11 @@ CF_CLI_HELP(GRACEFUL, restart, [[Shut the daemon down for graceful restart]])
CF_CLI(GRACEFUL RESTART,,, [[Shut the daemon down for graceful restart]]) CF_CLI(GRACEFUL RESTART,,, [[Shut the daemon down for graceful restart]])
{ cmd_graceful_restart(); } ; { cmd_graceful_restart(); } ;
CF_CLI(SHOW THREADS,,, [[Write out thread information]])
{ cmd_show_threads(0); } ;
CF_CLI(SHOW THREADS ALL,,, [[Write out thread and IO loop information]])
{ cmd_show_threads(1); } ;
cfg_name: cfg_name:
/* empty */ { $$ = NULL; } /* empty */ { $$ = NULL; }

View File

@ -70,6 +70,12 @@ domain_free(struct domain_generic *dg)
xfree(dg); xfree(dg);
} }
const char *
domain_name(struct domain_generic *dg)
{
return dg->name;
}
uint dg_order(struct domain_generic *dg) uint dg_order(struct domain_generic *dg)
{ {
return dg->order; return dg->order;

View File

@ -18,6 +18,7 @@
#include "lib/buffer.h" #include "lib/buffer.h"
#include "lib/lists.h" #include "lib/lists.h"
#include "lib/locking.h"
#include "lib/resource.h" #include "lib/resource.h"
#include "lib/event.h" #include "lib/event.h"
#include "lib/timer.h" #include "lib/timer.h"
@ -26,9 +27,36 @@
#include "lib/io-loop.h" #include "lib/io-loop.h"
#include "sysdep/unix/io-loop.h" #include "sysdep/unix/io-loop.h"
#include "conf/conf.h" #include "conf/conf.h"
#include "nest/cli.h"
#define THREAD_STACK_SIZE 65536 /* To be lowered in near future */ #define THREAD_STACK_SIZE 65536 /* To be lowered in near future */
/*
* Nanosecond time for accounting purposes
*
* A fixed point on startup is set as zero, all other values are relative to that.
* Caution: this overflows after like 500 years or so. If you plan to run
* BIRD for such a long time, please implement some means of overflow prevention.
*/
static struct timespec ns_begin;
static void ns_init(void)
{
if (clock_gettime(CLOCK_MONOTONIC, &ns_begin))
bug("clock_gettime: %m");
}
static u64 ns_now(void)
{
struct timespec ts;
if (clock_gettime(CLOCK_MONOTONIC, &ts))
bug("clock_gettime: %m");
return (u64) (ts.tv_sec - ns_begin.tv_sec) * 1000000000 + ts.tv_nsec - ns_begin.tv_nsec;
}
/* /*
* Current thread context * Current thread context
*/ */
@ -59,6 +87,12 @@ birdloop_inside(struct birdloop *loop)
return 0; return 0;
} }
_Bool
birdloop_in_this_thread(struct birdloop *loop)
{
return pthread_equal(pthread_self(), loop->thread->thread_id);
}
void void
birdloop_flag(struct birdloop *loop, u32 flag) birdloop_flag(struct birdloop *loop, u32 flag)
{ {
@ -80,22 +114,10 @@ birdloop_process_flags(struct birdloop *loop)
return 0; return 0;
u32 flags = atomic_exchange_explicit(&loop->flags, 0, memory_order_acq_rel); u32 flags = atomic_exchange_explicit(&loop->flags, 0, memory_order_acq_rel);
if (!flags)
return 0;
loop->flag_handler->hook(loop->flag_handler, flags); loop->flag_handler->hook(loop->flag_handler, flags);
return !!flags;
}
static int
birdloop_run_events(struct birdloop *loop)
{
btime begin = current_time();
while (current_time() - begin < 5 MS)
{
if (!ev_run_list(&loop->event_list))
return 0;
times_update();
}
return 1; return 1;
} }
@ -160,7 +182,7 @@ pipe_kick(struct pipe *p)
while (1) { while (1) {
rv = write(p->fd[1], &v, sizeof(v)); rv = write(p->fd[1], &v, sizeof(v));
if ((rv >= 0) || (errno == EAGAIN)) if ((rv >= 0) || (errno == EAGAIN))
return; return;
if (errno != EINTR) if (errno != EINTR)
bug("wakeup write: %m"); bug("wakeup write: %m");
@ -176,19 +198,19 @@ pipe_pollin(struct pipe *p, struct pollfd *pfd)
} }
static inline void static inline void
wakeup_init(struct birdloop *loop) wakeup_init(struct bird_thread *loop)
{ {
pipe_new(&loop->wakeup); pipe_new(&loop->wakeup);
} }
static inline void static inline void
wakeup_drain(struct birdloop *loop) wakeup_drain(struct bird_thread *loop)
{ {
pipe_drain(&loop->wakeup); pipe_drain(&loop->wakeup);
} }
static inline void static inline void
wakeup_do_kick(struct birdloop *loop) wakeup_do_kick(struct bird_thread *loop)
{ {
pipe_kick(&loop->wakeup); pipe_kick(&loop->wakeup);
} }
@ -196,13 +218,16 @@ wakeup_do_kick(struct birdloop *loop)
static inline void static inline void
birdloop_do_ping(struct birdloop *loop) birdloop_do_ping(struct birdloop *loop)
{ {
if (atomic_fetch_add_explicit(&loop->ping_sent, 1, memory_order_acq_rel)) if (!loop->thread)
return;
if (atomic_fetch_add_explicit(&loop->thread->ping_sent, 1, memory_order_acq_rel))
return; return;
if (loop == birdloop_wakeup_masked) if (loop == birdloop_wakeup_masked)
birdloop_wakeup_masked_count++; birdloop_wakeup_masked_count++;
else else
wakeup_do_kick(loop); wakeup_do_kick(loop->thread);
} }
void void
@ -224,10 +249,6 @@ sockets_init(struct birdloop *loop)
{ {
init_list(&loop->sock_list); init_list(&loop->sock_list);
loop->sock_num = 0; loop->sock_num = 0;
BUFFER_INIT(loop->poll_sk, loop->pool, 4);
BUFFER_INIT(loop->poll_fd, loop->pool, 4);
loop->poll_changed = 1; /* add wakeup fd */
} }
static void static void
@ -237,7 +258,8 @@ sockets_add(struct birdloop *loop, sock *s)
loop->sock_num++; loop->sock_num++;
s->index = -1; s->index = -1;
loop->poll_changed = 1; if (loop->thread)
atomic_store_explicit(&loop->thread->poll_changed, 1, memory_order_release);
birdloop_ping(loop); birdloop_ping(loop);
} }
@ -255,19 +277,20 @@ sockets_remove(struct birdloop *loop, sock *s)
if (!enlisted(&s->n)) if (!enlisted(&s->n))
return; return;
/* Decouple the socket from the loop at all. */
rem_node(&s->n); rem_node(&s->n);
loop->sock_num--; loop->sock_num--;
if (loop->thread)
atomic_store_explicit(&loop->thread->poll_changed, 1, memory_order_release);
if (s->index >= 0) s->index = -1;
{
loop->poll_sk.data[s->index] = NULL; /* Close the filedescriptor. If it ever gets into the poll(), it just returns
s->index = -1; * POLLNVAL for this fd which then is ignored because nobody checks for
loop->poll_changed = 1; * that result. Or some other routine opens another fd, getting this number,
loop->close_scheduled = 1; * yet also in this case poll() at worst spuriously returns and nobody checks
birdloop_ping(loop); * for the result in this fd. No further precaution is needed. */
} close(s->fd);
else
close(s->fd);
} }
void void
@ -279,67 +302,30 @@ sk_stop(sock *s)
static inline uint sk_want_events(sock *s) static inline uint sk_want_events(sock *s)
{ return (s->rx_hook ? POLLIN : 0) | ((s->ttx != s->tpos) ? POLLOUT : 0); } { return (s->rx_hook ? POLLIN : 0) | ((s->ttx != s->tpos) ? POLLOUT : 0); }
/* static struct pollfd *
FIXME: this should be called from sock code sockets_prepare(struct birdloop *loop, struct pollfd *pfd, struct pollfd *end)
static void
sockets_update(struct birdloop *loop, sock *s)
{ {
if (s->index >= 0)
loop->poll_fd.data[s->index].events = sk_want_events(s);
}
*/
static void
sockets_prepare(struct birdloop *loop)
{
BUFFER_SET(loop->poll_sk, loop->sock_num + 1);
BUFFER_SET(loop->poll_fd, loop->sock_num + 1);
struct pollfd *pfd = loop->poll_fd.data;
sock **psk = loop->poll_sk.data;
uint i = 0;
node *n; node *n;
loop->pfd = pfd;
WALK_LIST(n, loop->sock_list) WALK_LIST(n, loop->sock_list)
{ {
sock *s = SKIP_BACK(sock, n, n); sock *s = SKIP_BACK(sock, n, n);
ASSERT(i < loop->sock_num); /* Out of space for pfds. Force reallocation. */
if (pfd >= end)
return NULL;
s->index = pfd - loop->pfd;
s->index = i;
*psk = s;
pfd->fd = s->fd; pfd->fd = s->fd;
pfd->events = sk_want_events(s); pfd->events = sk_want_events(s);
pfd->revents = 0; pfd->revents = 0;
pfd++; pfd++;
psk++;
i++;
} }
ASSERT(i == loop->sock_num); return pfd;
/* Add internal wakeup fd */
*psk = NULL;
pipe_pollin(&loop->wakeup, pfd);
loop->poll_changed = 0;
}
static void
sockets_close_fds(struct birdloop *loop)
{
struct pollfd *pfd = loop->poll_fd.data;
sock **psk = loop->poll_sk.data;
int poll_num = loop->poll_fd.used - 1;
int i;
for (i = 0; i < poll_num; i++)
if (psk[i] == NULL)
close(pfd[i].fd);
loop->close_scheduled = 0;
} }
int sk_read(sock *s, int revents); int sk_read(sock *s, int revents);
@ -348,54 +334,527 @@ int sk_write(sock *s);
static void static void
sockets_fire(struct birdloop *loop) sockets_fire(struct birdloop *loop)
{ {
struct pollfd *pfd = loop->poll_fd.data; struct pollfd *pfd = loop->pfd;
sock **psk = loop->poll_sk.data;
int poll_num = loop->poll_fd.used - 1;
times_update(); times_update();
/* Last fd is internal wakeup fd */ sock *s; node *n, *nxt;
if (pfd[poll_num].revents & POLLIN) WALK_LIST2_DELSAFE(s, n, nxt, loop->sock_list, n)
wakeup_drain(loop);
int i;
for (i = 0; i < poll_num; pfd++, psk++, i++)
{ {
int e = 1; if (s->index < 0)
if (! pfd->revents)
continue; continue;
if (pfd->revents & POLLNVAL) int rev = pfd[s->index].revents;
bug("poll: invalid fd %d", pfd->fd);
if (pfd->revents & POLLIN) if (!rev)
while (e && *psk && (*psk)->rx_hook) continue;
e = sk_read(*psk, pfd->revents);
e = 1; if (rev & POLLNVAL)
if (pfd->revents & POLLOUT) bug("poll: invalid fd %d", s->fd);
int e = 1;
if (rev & POLLIN)
while (e && s->rx_hook)
e = sk_read(s, rev);
if (rev & POLLOUT)
{ {
loop->poll_changed = 1; atomic_store_explicit(&loop->thread->poll_changed, 1, memory_order_release);
while (e && *psk) while (e = sk_write(s))
e = sk_write(*psk); ;
} }
} }
} }
/*
* Threads
*/
DEFINE_DOMAIN(resource);
static DOMAIN(resource) birdloop_domain;
static list birdloop_pickup;
static list bird_thread_pickup;
static _Thread_local struct bird_thread *this_thread;
static void *
bird_thread_main(void *arg)
{
struct bird_thread *thr = this_thread = arg;
rcu_thread_start(&thr->rcu);
synchronize_rcu();
tmp_init(thr->pool);
init_list(&thr->loops);
u32 refresh_sockets = 1;
struct pollfd *pfd, *end;
while (1)
{
/* Wakeup at least once a minute. */
int timeout = 60000;
/* Pickup new loops */
LOCK_DOMAIN(resource, birdloop_domain);
if (!EMPTY_LIST(birdloop_pickup))
{
struct birdloop *loop = SKIP_BACK(struct birdloop, n, HEAD(birdloop_pickup));
rem_node(&loop->n);
UNLOCK_DOMAIN(resource, birdloop_domain);
add_tail(&thr->loops, &loop->n);
birdloop_enter(loop);
loop->thread = thr;
if (!EMPTY_LIST(loop->sock_list))
refresh_sockets = 1;
birdloop_leave(loop);
/* If there are more loops to be picked up, wakeup the next thread */
LOCK_DOMAIN(resource, birdloop_domain);
rem_node(&thr->n);
add_tail(&bird_thread_pickup, &thr->n);
if (!EMPTY_LIST(birdloop_pickup))
wakeup_do_kick(SKIP_BACK(struct bird_thread, n, HEAD(bird_thread_pickup)));
}
UNLOCK_DOMAIN(resource, birdloop_domain);
struct birdloop *loop; node *nn;
WALK_LIST2(loop, nn, thr->loops, n)
{
birdloop_enter(loop);
u64 after_enter = ns_now();
timer *t;
times_update();
timers_fire(&loop->time, 0);
int again = birdloop_process_flags(loop) + ev_run_list(&loop->event_list);
#if 0
if (loop->n.next->next)
__builtin_prefetch(SKIP_BACK(struct birdloop, n, loop->n.next)->time.domain);
#endif
if (again)
timeout = MIN(0, timeout);
else if (t = timers_first(&loop->time))
timeout = MIN(((tm_remains(t) TO_MS) + 1), timeout);
u64 before_leave = ns_now();
loop->total_time_spent_ns += (before_leave - after_enter);
birdloop_leave(loop);
ev_run_list(&thr->priority_events);
}
refresh_sockets += atomic_exchange_explicit(&thr->poll_changed, 0, memory_order_acq_rel);
if (!refresh_sockets && ((timeout < 0) || (timeout > 5000)))
flush_local_pages();
while (refresh_sockets)
{
sock_retry:;
end = (pfd = thr->pfd) + thr->pfd_max;
/* Add internal wakeup fd */
pipe_pollin(&thr->wakeup, pfd);
pfd++;
WALK_LIST2(loop, nn, thr->loops, n)
{
birdloop_enter(loop);
pfd = sockets_prepare(loop, pfd, end);
birdloop_leave(loop);
if (!pfd)
{
mb_free(thr->pfd);
thr->pfd = mb_alloc(thr->pool, sizeof(struct pollfd) * (thr->pfd_max *= 2));
goto sock_retry;
}
}
refresh_sockets = 0;
}
poll_retry:;
int rv = poll(thr->pfd, pfd - thr->pfd, timeout);
if (rv < 0)
{
if (errno == EINTR || errno == EAGAIN)
goto poll_retry;
bug("poll in %p: %m", thr);
}
/* Drain wakeup fd */
if (thr->pfd[0].revents & POLLIN)
{
ASSERT_DIE(rv > 0);
rv--;
wakeup_drain(thr);
}
atomic_exchange_explicit(&thr->ping_sent, 0, memory_order_acq_rel);
if (!rv && !atomic_exchange_explicit(&thr->run_cleanup, 0, memory_order_acq_rel))
continue;
/* Process stops and regular sockets */
node *nxt;
WALK_LIST2_DELSAFE(loop, nn, nxt, thr->loops, n)
{
birdloop_enter(loop);
if (loop->stopped)
{
/* Flush remaining events */
ASSERT_DIE(!ev_run_list(&loop->event_list));
/* Drop timers */
timer *t;
while (t = timers_first(&loop->time))
tm_stop(t);
/* No sockets allowed */
ASSERT_DIE(EMPTY_LIST(loop->sock_list));
/* Declare loop stopped */
rem_node(&loop->n);
birdloop_leave(loop);
loop->stopped(loop->stop_data);
/* Birdloop already left */
continue;
}
else if (rv)
sockets_fire(loop);
birdloop_leave(loop);
}
}
}
static void
bird_thread_cleanup(void *_thr)
{
struct bird_thread *thr = _thr;
ASSERT_DIE(birdloop_inside(&main_birdloop));
/* Thread attributes no longer needed */
pthread_attr_destroy(&thr->thread_attr);
/* Free all remaining memory */
rfree(thr->pool);
}
static struct bird_thread *
bird_thread_start(void)
{
ASSERT_DIE(birdloop_inside(&main_birdloop));
pool *p = rp_new(&root_pool, "Thread");
struct bird_thread *thr = mb_allocz(p, sizeof(*thr));
thr->pool = p;
thr->pfd = mb_alloc(p, sizeof(struct pollfd) * (thr->pfd_max = 16));
thr->cleanup_event = (event) { .hook = bird_thread_cleanup, .data = thr, };
atomic_store_explicit(&thr->ping_sent, 0, memory_order_relaxed);
wakeup_init(thr);
ev_init_list(&thr->priority_events, NULL, "Thread direct event list");
LOCK_DOMAIN(resource, birdloop_domain);
add_tail(&bird_thread_pickup, &thr->n);
UNLOCK_DOMAIN(resource, birdloop_domain);
int e = 0;
if (e = pthread_attr_init(&thr->thread_attr))
die("pthread_attr_init() failed: %M", e);
/* We don't have to worry about thread stack size so much.
if (e = pthread_attr_setstacksize(&thr->thread_attr, THREAD_STACK_SIZE))
die("pthread_attr_setstacksize(%u) failed: %M", THREAD_STACK_SIZE, e);
*/
if (e = pthread_attr_setdetachstate(&thr->thread_attr, PTHREAD_CREATE_DETACHED))
die("pthread_attr_setdetachstate(PTHREAD_CREATE_DETACHED) failed: %M", e);
if (e = pthread_create(&thr->thread_id, &thr->thread_attr, bird_thread_main, thr))
die("pthread_create() failed: %M", e);
return thr;
}
static struct birdloop *thread_dropper;
static event *thread_dropper_event;
static uint thread_dropper_goal;
static void
bird_thread_shutdown(void * _ UNUSED)
{
LOCK_DOMAIN(resource, birdloop_domain);
int dif = list_length(&bird_thread_pickup) - thread_dropper_goal;
struct birdloop *tdl_stop = NULL;
if (dif > 0)
ev_send_loop(thread_dropper, thread_dropper_event);
else
{
tdl_stop = thread_dropper;
thread_dropper = NULL;
}
UNLOCK_DOMAIN(resource, birdloop_domain);
log(L_INFO "Thread pickup size differs from dropper goal by %d%s", dif, tdl_stop ? ", stopping" : "");
if (tdl_stop)
{
birdloop_stop_self(tdl_stop, NULL, NULL);
return;
}
struct bird_thread *thr = this_thread;
/* Leave the thread-picker list to get no more loops */
LOCK_DOMAIN(resource, birdloop_domain);
rem_node(&thr->n);
/* Drop loops including the thread dropper itself */
while (!EMPTY_LIST(thr->loops))
{
/* Remove loop from this thread's list */
struct birdloop *loop = HEAD(thr->loops);
rem_node(&loop->n);
UNLOCK_DOMAIN(resource, birdloop_domain);
/* Unset loop's thread */
if (birdloop_inside(loop))
loop->thread = NULL;
else
{
birdloop_enter(loop);
loop->thread = NULL;
birdloop_leave(loop);
}
/* Put loop into pickup list */
LOCK_DOMAIN(resource, birdloop_domain);
add_tail(&birdloop_pickup, &loop->n);
}
/* Let others know about new loops */
if (!EMPTY_LIST(birdloop_pickup))
wakeup_do_kick(SKIP_BACK(struct bird_thread, n, HEAD(bird_thread_pickup)));
UNLOCK_DOMAIN(resource, birdloop_domain);
/* Leave the thread-dropper loop as we aren't going to return. */
birdloop_leave(thread_dropper);
/* Local pages not needed anymore */
flush_local_pages();
/* Unregister from RCU */
rcu_thread_stop(&thr->rcu);
/* Request thread cleanup from main loop */
ev_send_loop(&main_birdloop, &thr->cleanup_event);
/* Exit! */
pthread_exit(NULL);
}
void
bird_thread_commit(struct config *new, struct config *old UNUSED)
{
ASSERT_DIE(birdloop_inside(&main_birdloop));
if (new->shutdown)
return;
if (!new->thread_count)
new->thread_count = 1;
while (1)
{
LOCK_DOMAIN(resource, birdloop_domain);
int dif = list_length(&bird_thread_pickup) - (thread_dropper_goal = new->thread_count);
_Bool thread_dropper_running = !!thread_dropper;
UNLOCK_DOMAIN(resource, birdloop_domain);
if (dif < 0)
{
bird_thread_start();
continue;
}
if ((dif > 0) && !thread_dropper_running)
{
struct birdloop *tdl = birdloop_new(&root_pool, DOMAIN_ORDER(control), "Thread dropper");
event *tde = ev_new_init(tdl->pool, bird_thread_shutdown, NULL);
LOCK_DOMAIN(resource, birdloop_domain);
thread_dropper = tdl;
thread_dropper_event = tde;
UNLOCK_DOMAIN(resource, birdloop_domain);
ev_send_loop(thread_dropper, thread_dropper_event);
}
return;
}
}
DEFINE_DOMAIN(control);
struct bird_thread_show_data {
cli *cli;
pool *pool;
DOMAIN(control) lock;
uint total;
uint done;
u8 show_loops;
};
static void
bird_thread_show_cli_cont(struct cli *c UNUSED)
{
/* Explicitly do nothing to prevent CLI from trying to parse another command. */
}
static int
bird_thread_show_cli_cleanup(struct cli *c UNUSED)
{
return 1; /* Defer the cleanup until the writeout is finished. */
}
static void
bird_thread_show(void *data)
{
struct bird_thread_show_data *tsd = data;
LOCK_DOMAIN(control, tsd->lock);
if (tsd->show_loops)
cli_printf(tsd->cli, -1026, "Thread %p", this_thread);
u64 total_time_ns = 0;
struct birdloop *loop;
WALK_LIST(loop, this_thread->loops)
{
if (tsd->show_loops)
cli_printf(tsd->cli, -1026, " Loop %s time: %t", domain_name(loop->time.domain), loop->total_time_spent_ns NS);
total_time_ns += loop->total_time_spent_ns;
}
tsd->done++;
int last = (tsd->done == tsd->total);
if (last)
{
tsd->cli->cont = NULL;
tsd->cli->cleanup = NULL;
}
if (tsd->show_loops)
cli_printf(tsd->cli, (last ? 1 : -1) * 1026, " Total time: %t", total_time_ns NS);
else
cli_printf(tsd->cli, (last ? 1 : -1) * 1026, "Thread %p time %t", this_thread, total_time_ns NS);
UNLOCK_DOMAIN(control, tsd->lock);
if (last)
{
the_bird_lock();
LOCK_DOMAIN(resource, birdloop_domain);
if (!EMPTY_LIST(birdloop_pickup))
if (tsd->show_loops)
{
cli_printf(tsd->cli, -1026, "Unassigned loops");
WALK_LIST(loop, birdloop_pickup)
cli_printf(tsd->cli, -1026, " Loop %s time: %t", domain_name(loop->time.domain), loop->total_time_spent_ns NS);
}
else
{
uint count = 0;
u64 total_time_ns = 0;
WALK_LIST(loop, birdloop_pickup)
{
count++;
total_time_ns += loop->total_time_spent_ns;
}
cli_printf(tsd->cli, -1026, "Unassigned loops: %d, total time %t", count, total_time_ns NS);
}
UNLOCK_DOMAIN(resource, birdloop_domain);
cli_write_trigger(tsd->cli);
DOMAIN_FREE(control, tsd->lock);
rfree(tsd->pool);
the_bird_unlock();
}
}
void
cmd_show_threads(int show_loops)
{
pool *p = rp_new(&root_pool, "Show Threads");
struct bird_thread_show_data *tsd = mb_allocz(p, sizeof(struct bird_thread_show_data));
tsd->lock = DOMAIN_NEW(control, "Show Threads");
tsd->cli = this_cli;
tsd->pool = p;
tsd->show_loops = show_loops;
this_cli->cont = bird_thread_show_cli_cont;
this_cli->cleanup = bird_thread_show_cli_cleanup;
LOCK_DOMAIN(control, tsd->lock);
LOCK_DOMAIN(resource, birdloop_domain);
struct bird_thread *thr;
WALK_LIST(thr, bird_thread_pickup)
{
tsd->total++;
ev_send(&thr->priority_events, ev_new_init(p, bird_thread_show, tsd));
wakeup_do_kick(thr);
}
UNLOCK_DOMAIN(resource, birdloop_domain);
UNLOCK_DOMAIN(control, tsd->lock);
}
/* /*
* Birdloop * Birdloop
*/ */
struct birdloop main_birdloop; static struct bird_thread main_thread;
struct birdloop main_birdloop = { .thread = &main_thread, };
static void birdloop_enter_locked(struct birdloop *loop); static void birdloop_enter_locked(struct birdloop *loop);
void void
birdloop_init(void) birdloop_init(void)
{ {
wakeup_init(&main_birdloop); ns_init();
birdloop_domain = DOMAIN_NEW(resource, "Loop Pickup");
init_list(&birdloop_pickup);
init_list(&bird_thread_pickup);
wakeup_init(main_birdloop.thread);
main_birdloop.time.domain = the_bird_domain.the_bird; main_birdloop.time.domain = the_bird_domain.the_bird;
main_birdloop.time.loop = &main_birdloop; main_birdloop.time.loop = &main_birdloop;
@ -406,8 +865,6 @@ birdloop_init(void)
birdloop_enter_locked(&main_birdloop); birdloop_enter_locked(&main_birdloop);
} }
static void *birdloop_main(void *arg);
struct birdloop * struct birdloop *
birdloop_new(pool *pp, uint order, const char *name) birdloop_new(pool *pp, uint order, const char *name)
{ {
@ -422,24 +879,14 @@ birdloop_new(pool *pp, uint order, const char *name)
birdloop_enter(loop); birdloop_enter(loop);
wakeup_init(loop);
ev_init_list(&loop->event_list, loop, name); ev_init_list(&loop->event_list, loop, name);
timers_init(&loop->time, p); timers_init(&loop->time, p);
sockets_init(loop); sockets_init(loop);
int e = 0; LOCK_DOMAIN(resource, birdloop_domain);
add_tail(&birdloop_pickup, &loop->n);
if (e = pthread_attr_init(&loop->thread_attr)) wakeup_do_kick(SKIP_BACK(struct bird_thread, n, HEAD(bird_thread_pickup)));
die("pthread_attr_init() failed: %M", e); UNLOCK_DOMAIN(resource, birdloop_domain);
if (e = pthread_attr_setstacksize(&loop->thread_attr, THREAD_STACK_SIZE))
die("pthread_attr_setstacksize(%u) failed: %M", THREAD_STACK_SIZE, e);
if (e = pthread_attr_setdetachstate(&loop->thread_attr, PTHREAD_CREATE_DETACHED))
die("pthread_attr_setdetachstate(PTHREAD_CREATE_DETACHED) failed: %M", e);
if (e = pthread_create(&loop->thread_id, &loop->thread_attr, birdloop_main, loop))
die("pthread_create() failed: %M", e);
birdloop_leave(loop); birdloop_leave(loop);
@ -451,7 +898,11 @@ birdloop_do_stop(struct birdloop *loop, void (*stopped)(void *data), void *data)
{ {
loop->stopped = stopped; loop->stopped = stopped;
loop->stop_data = data; loop->stop_data = data;
wakeup_do_kick(loop); if (loop->thread)
{
atomic_store_explicit(&loop->thread->run_cleanup, 1, memory_order_release);
wakeup_do_kick(loop->thread);
}
} }
void void
@ -475,10 +926,7 @@ void
birdloop_free(struct birdloop *loop) birdloop_free(struct birdloop *loop)
{ {
ASSERT_DIE(loop->links == 0); ASSERT_DIE(loop->links == 0);
ASSERT_DIE(pthread_equal(pthread_self(), loop->thread_id)); ASSERT_DIE(birdloop_in_this_thread(loop));
rcu_birdloop_stop(&loop->rcu);
pthread_attr_destroy(&loop->thread_attr);
domain_free(loop->time.domain); domain_free(loop->time.domain);
rfree(loop->pool); rfree(loop->pool);
@ -541,7 +989,7 @@ birdloop_unmask_wakeups(struct birdloop *loop)
ASSERT_DIE(birdloop_wakeup_masked == loop); ASSERT_DIE(birdloop_wakeup_masked == loop);
birdloop_wakeup_masked = NULL; birdloop_wakeup_masked = NULL;
if (birdloop_wakeup_masked_count) if (birdloop_wakeup_masked_count)
wakeup_do_kick(loop); wakeup_do_kick(loop->thread);
birdloop_wakeup_masked_count = 0; birdloop_wakeup_masked_count = 0;
} }
@ -560,85 +1008,6 @@ birdloop_unlink(struct birdloop *loop)
loop->links--; loop->links--;
} }
static void *
birdloop_main(void *arg)
{
struct birdloop *loop = arg;
timer *t;
int rv, timeout;
rcu_birdloop_start(&loop->rcu);
btime loop_begin = current_time();
tmp_init(loop->pool);
birdloop_enter(loop);
while (1)
{
timers_fire(&loop->time, 0);
if (birdloop_process_flags(loop) + birdloop_run_events(loop))
timeout = 0;
else if (t = timers_first(&loop->time))
timeout = (tm_remains(t) TO_MS) + 1;
else
timeout = -1;
if (loop->poll_changed)
sockets_prepare(loop);
else
if ((timeout < 0) || (timeout > 5000))
flush_local_pages();
btime duration = current_time() - loop_begin;
if (duration > config->watchdog_warning)
log(L_WARN "I/O loop cycle took %d ms", (int) (duration TO_MS));
birdloop_leave(loop);
try:
rv = poll(loop->poll_fd.data, loop->poll_fd.used, timeout);
if (rv < 0)
{
if (errno == EINTR || errno == EAGAIN)
goto try;
bug("poll: %m");
}
birdloop_enter(loop);
if (loop->close_scheduled)
sockets_close_fds(loop);
if (loop->stopped)
break;
loop_begin = current_time();
if (rv && !loop->poll_changed)
sockets_fire(loop);
atomic_exchange_explicit(&loop->ping_sent, 0, memory_order_acq_rel);
}
/* Flush remaining events */
ASSERT_DIE(!ev_run_list(&loop->event_list));
/* Drop timers */
while (t = timers_first(&loop->time))
tm_stop(t);
/* No sockets allowed */
ASSERT_DIE(EMPTY_LIST(loop->sock_list));
ASSERT_DIE(loop->sock_num == 0);
birdloop_leave(loop);
loop->stopped(loop->stop_data);
flush_local_pages();
return NULL;
}
void void
birdloop_yield(void) birdloop_yield(void)
{ {

View File

@ -21,26 +21,16 @@ void pipe_kick(struct pipe *);
struct birdloop struct birdloop
{ {
node n;
pool *pool; pool *pool;
struct timeloop time; struct timeloop time;
event_list event_list; event_list event_list;
list sock_list; list sock_list;
uint sock_num; int sock_num;
BUFFER(sock *) poll_sk;
BUFFER(struct pollfd) poll_fd;
u8 poll_changed;
u8 close_scheduled;
uint ping_pending; uint ping_pending;
_Atomic u32 ping_sent;
struct pipe wakeup;
pthread_t thread_id;
pthread_attr_t thread_attr;
struct rcu_birdloop rcu;
uint links; uint links;
@ -51,6 +41,36 @@ struct birdloop
void *stop_data; void *stop_data;
struct birdloop *prev_loop; struct birdloop *prev_loop;
struct bird_thread *thread;
struct pollfd *pfd;
u64 total_time_spent_ns;
};
struct bird_thread
{
node n;
struct pollfd *pfd;
uint pfd_max;
_Atomic u32 ping_sent;
_Atomic u32 run_cleanup;
_Atomic u32 poll_changed;
struct pipe wakeup;
event_list priority_events;
pthread_t thread_id;
pthread_attr_t thread_attr;
struct rcu_thread rcu;
list loops;
pool *pool;
event cleanup_event;
}; };
#endif #endif

View File

@ -2254,7 +2254,7 @@ io_loop(void)
} }
/* A hack to reload main io_loop() when something has changed asynchronously. */ /* A hack to reload main io_loop() when something has changed asynchronously. */
pipe_pollin(&main_birdloop.wakeup, &pfd[0]); pipe_pollin(&main_birdloop.thread->wakeup, &pfd[0]);
nfds = 1; nfds = 1;
@ -2332,8 +2332,8 @@ io_loop(void)
if (pfd[0].revents & POLLIN) if (pfd[0].revents & POLLIN)
{ {
/* IO loop reload requested */ /* IO loop reload requested */
pipe_drain(&main_birdloop.wakeup); pipe_drain(&main_birdloop.thread->wakeup);
atomic_exchange_explicit(&main_birdloop.ping_sent, 0, memory_order_acq_rel); atomic_exchange_explicit(&main_birdloop.thread->ping_sent, 0, memory_order_acq_rel);
continue; continue;
} }

View File

@ -200,9 +200,11 @@ sysdep_preconfig(struct config *c)
} }
int int
sysdep_commit(struct config *new, struct config *old UNUSED) sysdep_commit(struct config *new, struct config *old)
{ {
log_switch(0, &new->logfiles, new->syslog_name); log_switch(0, &new->logfiles, new->syslog_name);
bird_thread_commit(new, old);
return 0; return 0;
} }

View File

@ -16,6 +16,7 @@ struct pool;
struct iface; struct iface;
struct birdsock; struct birdsock;
struct rfile; struct rfile;
struct config;
/* main.c */ /* main.c */
@ -32,6 +33,8 @@ void cmd_reconfig_undo(void);
void cmd_reconfig_status(void); void cmd_reconfig_status(void);
void cmd_shutdown(void); void cmd_shutdown(void);
void cmd_graceful_restart(void); void cmd_graceful_restart(void);
void cmd_show_threads(int);
void bird_thread_commit(struct config *new, struct config *old);
#define UNIX_DEFAULT_CONFIGURE_TIMEOUT 300 #define UNIX_DEFAULT_CONFIGURE_TIMEOUT 300

View File

@ -557,7 +557,14 @@ void cmd_reconfig_undo_notify(void) {}
#include "lib/net.h" #include "lib/net.h"
#include "conf/conf.h" #include "conf/conf.h"
void sysdep_preconfig(struct config *c UNUSED) {} void sysdep_preconfig(struct config *c UNUSED) {}
int sysdep_commit(struct config *new UNUSED, struct config *old UNUSED) { return 0; }
void bird_thread_commit(struct config *new, struct config *old);
int sysdep_commit(struct config *new, struct config *old)
{
bird_thread_commit(new, old);
return 0;
}
void sysdep_shutdown_done(void) {} void sysdep_shutdown_done(void) {}
#include "nest/cli.h" #include "nest/cli.h"

View File

@ -28,7 +28,6 @@ uint bt_naive_pow(uint base, uint power);
void bt_bytes_to_hex(char *buf, const byte *in_data, size_t size); void bt_bytes_to_hex(char *buf, const byte *in_data, size_t size);
void bt_bird_init(void); void bt_bird_init(void);
void bt_bird_cleanup(void);
struct config *bt_config_parse(const char *cfg); struct config *bt_config_parse(const char *cfg);
struct config *bt_config_file_parse(const char *filepath); struct config *bt_config_file_parse(const char *filepath);