*: Convert thread_cancelXXX to event_cancelXXX

Modify the code base so that thread_cancel becomes event_cancel

Signed-off-by: Donald Sharp <sharpd@nvidia.com>
This commit is contained in:
Donald Sharp 2022-12-10 09:08:37 -05:00
parent 907a2395f4
commit 332beb64b8
37 changed files with 117 additions and 118 deletions

View File

@ -306,8 +306,8 @@ babel_clean_routing_process(void)
babel_interface_close_all();
/* cancel events */
thread_cancel(&babel_routing_process->t_read);
thread_cancel(&babel_routing_process->t_update);
event_cancel(&babel_routing_process->t_read);
event_cancel(&babel_routing_process->t_update);
distribute_list_delete(&babel_routing_process->distribute_ctx);
XFREE(MTYPE_BABEL, babel_routing_process);
@ -485,7 +485,7 @@ static void
babel_set_timer(struct timeval *timeout)
{
long msecs = timeout->tv_sec * 1000 + timeout->tv_usec / 1000;
thread_cancel(&(babel_routing_process->t_update));
event_cancel(&(babel_routing_process->t_update));
event_add_timer_msec(master, babel_main_loop, NULL, msecs,
&babel_routing_process->t_update);
}

View File

@ -132,7 +132,7 @@ void control_shutdown(void)
{
struct bfd_control_socket *bcs;
thread_cancel(&bglobal.bg_csockev);
event_cancel(&bglobal.bg_csockev);
socket_close(&bglobal.bg_csock);
@ -185,8 +185,8 @@ static void control_free(struct bfd_control_socket *bcs)
struct bfd_control_queue *bcq;
struct bfd_notify_peer *bnp;
thread_cancel(&(bcs->bcs_ev));
thread_cancel(&(bcs->bcs_outev));
event_cancel(&(bcs->bcs_ev));
event_cancel(&(bcs->bcs_outev));
close(bcs->bcs_sd);
@ -292,7 +292,7 @@ static int control_queue_dequeue(struct bfd_control_socket *bcs)
return 1;
empty_list:
thread_cancel(&(bcs->bcs_outev));
event_cancel(&(bcs->bcs_outev));
bcs->bcs_bout = NULL;
return 0;
}

View File

@ -24,7 +24,7 @@
#define BGP_EVENT_FLUSH(P) \
do { \
assert(peer); \
thread_cancel_event_ready(bm->master, (P)); \
event_cancel_event_ready(bm->master, (P)); \
} while (0)
#define BGP_UPDATE_GROUP_TIMER_ON(T, F) \

View File

@ -65,7 +65,7 @@ void bgp_writes_off(struct peer *peer)
struct frr_pthread *fpt = bgp_pth_io;
assert(fpt->running);
thread_cancel_async(fpt->master, &peer->t_write, NULL);
event_cancel_async(fpt->master, &peer->t_write, NULL);
THREAD_OFF(peer->t_generate_updgrp_packets);
UNSET_FLAG(peer->thread_flags, PEER_THREAD_WRITES_ON);
@ -96,7 +96,7 @@ void bgp_reads_off(struct peer *peer)
struct frr_pthread *fpt = bgp_pth_io;
assert(fpt->running);
thread_cancel_async(fpt->master, &peer->t_read, NULL);
event_cancel_async(fpt->master, &peer->t_read, NULL);
THREAD_OFF(peer->t_process_packet);
THREAD_OFF(peer->t_process_packet_error);

View File

@ -1202,7 +1202,7 @@ static void lptest_stop(void)
}
if (tcb->event_thread)
thread_cancel(&tcb->event_thread);
event_cancel(&tcb->event_thread);
lpt_inprogress = false;
}
@ -1491,7 +1491,7 @@ static void lptest_delete(void *val)
}
if (tcb->event_thread)
thread_cancel(&tcb->event_thread);
event_cancel(&tcb->event_thread);
memset(tcb, 0, sizeof(*tcb));

View File

@ -1126,7 +1126,7 @@ static void peer_free(struct peer *peer)
bgp_timer_set(peer);
bgp_reads_off(peer);
bgp_writes_off(peer);
thread_cancel_event_ready(bm->master, peer);
event_cancel_event_ready(bm->master, peer);
FOREACH_AFI_SAFI (afi, safi)
THREAD_OFF(peer->t_revalidate_all[afi][safi]);
assert(!peer->t_write);
@ -2516,7 +2516,7 @@ int peer_delete(struct peer *peer)
bgp_keepalives_off(peer);
bgp_reads_off(peer);
bgp_writes_off(peer);
thread_cancel_event_ready(bm->master, peer);
event_cancel_event_ready(bm->master, peer);
FOREACH_AFI_SAFI (afi, safi)
THREAD_OFF(peer->t_revalidate_all[afi][safi]);
assert(!CHECK_FLAG(peer->thread_flags, PEER_THREAD_WRITES_ON));

View File

@ -136,7 +136,7 @@ Mapping the general names used in the figure to specific FRR functions:
- ``task`` is ``struct event *``
- ``fetch`` is ``thread_fetch()``
- ``exec()`` is ``thread_call``
- ``cancel()`` is ``thread_cancel()``
- ``cancel()`` is ``event_cancel()``
- ``schedule()`` is any of the various task-specific ``event_add_*`` functions
Adding tasks is done with various task-specific function-like macros. These
@ -228,7 +228,7 @@ well as *any other pthread*. This serves as the basis for inter-thread
communication and boils down to a slightly more complicated method of message
passing, where the messages are the regular task events as used in the
event-driven model. The only difference is thread cancellation, which requires
calling ``thread_cancel_async()`` instead of ``thread_cancel`` to cancel a task
calling ``event_cancel_async()`` instead of ``event_cancel`` to cancel a task
currently scheduled on a ``threadmaster`` belonging to a different pthread.
This is necessary to avoid race conditions in the specific case where one
pthread wants to guarantee that a task on another pthread is cancelled before

View File

@ -150,8 +150,8 @@ Example::
frr_libfrr:frr_pthread_stop (loglevel: TRACE_DEBUG_LINE (13)) (type: tracepoint)
frr_libfrr:frr_pthread_run (loglevel: TRACE_DEBUG_LINE (13)) (type: tracepoint)
frr_libfrr:thread_call (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:thread_cancel_async (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:thread_cancel (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:event_cancel_async (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:event_cancel (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:schedule_write (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:schedule_read (loglevel: TRACE_INFO (6)) (type: tracepoint)
frr_libfrr:schedule_event (loglevel: TRACE_INFO (6)) (type: tracepoint)

View File

@ -111,7 +111,7 @@ void eigrp_distribute_update(struct distribute_ctx *ctx,
// TODO: check Graceful restart after 10sec
/* cancel GR scheduled */
thread_cancel(&(e->t_distribute));
event_cancel(&(e->t_distribute));
/* schedule Graceful restart for whole process in 10sec */
event_add_timer(master, eigrp_distribute_timer_process, e, (10),
@ -186,7 +186,7 @@ void eigrp_distribute_update(struct distribute_ctx *ctx,
// TODO: check Graceful restart after 10sec
/* Cancel GR scheduled */
thread_cancel(&(ei->t_distribute));
event_cancel(&(ei->t_distribute));
/* schedule Graceful restart for interface in 10sec */
event_add_timer(master, eigrp_distribute_timer_interface, ei, 10,
&ei->t_distribute);

View File

@ -360,7 +360,7 @@ void eigrp_if_stream_unset(struct eigrp_interface *ei)
if (ei->on_write_q) {
listnode_delete(eigrp->oi_write_q, ei);
if (list_isempty(eigrp->oi_write_q))
thread_cancel(&(eigrp->t_write));
event_cancel(&(eigrp->t_write));
ei->on_write_q = 0;
}
}
@ -422,7 +422,7 @@ void eigrp_if_free(struct eigrp_interface *ei, int source)
struct eigrp *eigrp = ei->eigrp;
if (source == INTERFACE_DOWN_BY_VTY) {
thread_cancel(&ei->t_hello);
event_cancel(&ei->t_hello);
eigrp_hello_send(ei, EIGRP_HELLO_GRACEFUL_SHUTDOWN, NULL);
}

View File

@ -164,7 +164,7 @@ void eigrp_nbr_delete(struct eigrp_neighbor *nbr)
eigrp_topology_neighbor_down(nbr->ei->eigrp, nbr);
/* Cancel all events. */ /* Thread lookup cost would be negligible. */
thread_cancel_event(master, nbr);
event_cancel_event(master, nbr);
eigrp_fifo_free(nbr->multicast_queue);
eigrp_fifo_free(nbr->retrans_queue);
THREAD_OFF(nbr->t_holddown);

View File

@ -929,7 +929,7 @@ void isis_circuit_down(struct isis_circuit *circuit)
circuit->snd_stream = NULL;
}
thread_cancel_event(master, circuit);
event_cancel_event(master, circuit);
return;
}

View File

@ -548,7 +548,7 @@ void isis_area_destroy(struct isis_area *area)
THREAD_OFF(area->t_lsp_refresh[1]);
THREAD_OFF(area->t_rlfa_rib_update);
thread_cancel_event(master, area);
event_cancel_event(master, area);
listnode_delete(area->isis->area_list, area);

View File

@ -97,7 +97,7 @@ static void agentx_events_update(void)
struct event **thr;
int fd, thr_fd;
thread_cancel(&timeout_thr);
event_cancel(&timeout_thr);
FD_ZERO(&fds);
snmp_select_info(&maxfd, &fds, &timeout, &block);
@ -119,7 +119,7 @@ static void agentx_events_update(void)
if (thr_fd == fd) {
struct listnode *nextln = listnextnode(ln);
if (!FD_ISSET(fd, &fds)) {
thread_cancel(thr);
event_cancel(thr);
XFREE(MTYPE_TMP, thr);
list_delete_node(events, ln);
}
@ -142,7 +142,7 @@ static void agentx_events_update(void)
while (ln) {
struct listnode *nextln = listnextnode(ln);
thr = listgetdata(ln);
thread_cancel(thr);
event_cancel(thr);
XFREE(MTYPE_TMP, thr);
list_delete_node(events, ln);
ln = nextln;

View File

@ -38,7 +38,7 @@ struct cancel_req {
};
/* Flags for task cancellation */
#define THREAD_CANCEL_FLAG_READY 0x01
#define EVENT_CANCEL_FLAG_READY 0x01
static int thread_timer_cmp(const struct event *a, const struct event *b)
{
@ -1169,8 +1169,8 @@ void _event_add_event(const struct xref_threadsched *xref,
* - POLLIN
* - POLLOUT
*/
static void thread_cancel_rw(struct thread_master *master, int fd, short state,
int idx_hint)
static void event_cancel_rw(struct thread_master *master, int fd, short state,
int idx_hint)
{
bool found = false;
@ -1267,7 +1267,7 @@ static void cancel_arg_helper(struct thread_master *master,
}
/* If requested, stop here and ignore io and timers */
if (CHECK_FLAG(cr->flags, THREAD_CANCEL_FLAG_READY))
if (CHECK_FLAG(cr->flags, EVENT_CANCEL_FLAG_READY))
return;
/* Check the io tasks */
@ -1283,7 +1283,7 @@ static void cancel_arg_helper(struct thread_master *master,
fd = pfd->fd;
/* Found a match to cancel: clean up fd arrays */
thread_cancel_rw(master, pfd->fd, pfd->events, i);
event_cancel_rw(master, pfd->fd, pfd->events, i);
/* Clean up thread arrays */
master->read[fd] = NULL;
@ -1328,7 +1328,7 @@ static void cancel_arg_helper(struct thread_master *master,
* @param master the thread master to process
* @REQUIRE master->mtx
*/
static void do_thread_cancel(struct thread_master *master)
static void do_event_cancel(struct thread_master *master)
{
struct thread_list_head *list = NULL;
struct event **thread_array = NULL;
@ -1364,11 +1364,11 @@ static void do_thread_cancel(struct thread_master *master)
/* Determine the appropriate queue to cancel the thread from */
switch (thread->type) {
case THREAD_READ:
thread_cancel_rw(master, thread->u.fd, POLLIN, -1);
event_cancel_rw(master, thread->u.fd, POLLIN, -1);
thread_array = master->read;
break;
case THREAD_WRITE:
thread_cancel_rw(master, thread->u.fd, POLLOUT, -1);
event_cancel_rw(master, thread->u.fd, POLLOUT, -1);
thread_array = master->write;
break;
case THREAD_TIMER:
@ -1401,7 +1401,7 @@ static void do_thread_cancel(struct thread_master *master)
if (master->cancel_req)
list_delete_all_node(master->cancel_req);
/* Wake up any threads which may be blocked in thread_cancel_async() */
/* Wake up any threads which may be blocked in event_cancel_async() */
master->canceled = true;
pthread_cond_broadcast(&master->cancel_cond);
}
@ -1426,7 +1426,7 @@ static void cancel_event_helper(struct thread_master *m, void *arg, int flags)
frr_with_mutex (&m->mtx) {
cr->eventobj = arg;
listnode_add(m->cancel_req, cr);
do_thread_cancel(m);
do_event_cancel(m);
}
}
@ -1438,7 +1438,7 @@ static void cancel_event_helper(struct thread_master *m, void *arg, int flags)
* @param m the thread_master to cancel from
* @param arg the argument passed when creating the event
*/
void thread_cancel_event(struct thread_master *master, void *arg)
void event_cancel_event(struct thread_master *master, void *arg)
{
cancel_event_helper(master, arg, 0);
}
@ -1451,11 +1451,11 @@ void thread_cancel_event(struct thread_master *master, void *arg)
* @param m the thread_master to cancel from
* @param arg the argument passed when creating the event
*/
void thread_cancel_event_ready(struct thread_master *m, void *arg)
void event_cancel_event_ready(struct thread_master *m, void *arg)
{
/* Only cancel ready/event tasks */
cancel_event_helper(m, arg, THREAD_CANCEL_FLAG_READY);
cancel_event_helper(m, arg, EVENT_CANCEL_FLAG_READY);
}
/**
@ -1465,7 +1465,7 @@ void thread_cancel_event_ready(struct thread_master *m, void *arg)
*
* @param thread task to cancel
*/
void thread_cancel(struct event **thread)
void event_cancel(struct event **thread)
{
struct thread_master *master;
@ -1474,10 +1474,10 @@ void thread_cancel(struct event **thread)
master = (*thread)->master;
frrtrace(9, frr_libfrr, thread_cancel, master,
(*thread)->xref->funcname, (*thread)->xref->xref.file,
(*thread)->xref->xref.line, NULL, (*thread)->u.fd,
(*thread)->u.val, (*thread)->arg, (*thread)->u.sands.tv_sec);
frrtrace(9, frr_libfrr, event_cancel, master, (*thread)->xref->funcname,
(*thread)->xref->xref.file, (*thread)->xref->xref.line, NULL,
(*thread)->u.fd, (*thread)->u.val, (*thread)->arg,
(*thread)->u.sands.tv_sec);
assert(master->owner == pthread_self());
@ -1486,7 +1486,7 @@ void thread_cancel(struct event **thread)
XCALLOC(MTYPE_TMP, sizeof(struct cancel_req));
cr->thread = *thread;
listnode_add(master->cancel_req, cr);
do_thread_cancel(master);
do_event_cancel(master);
}
*thread = NULL;
@ -1516,19 +1516,19 @@ void thread_cancel(struct event **thread)
* @param thread pointer to thread to cancel
* @param eventobj the event
*/
void thread_cancel_async(struct thread_master *master, struct event **thread,
void *eventobj)
void event_cancel_async(struct thread_master *master, struct event **thread,
void *eventobj)
{
assert(!(thread && eventobj) && (thread || eventobj));
if (thread && *thread)
frrtrace(9, frr_libfrr, thread_cancel_async, master,
frrtrace(9, frr_libfrr, event_cancel_async, master,
(*thread)->xref->funcname, (*thread)->xref->xref.file,
(*thread)->xref->xref.line, NULL, (*thread)->u.fd,
(*thread)->u.val, (*thread)->arg,
(*thread)->u.sands.tv_sec);
else
frrtrace(9, frr_libfrr, thread_cancel_async, master, NULL, NULL,
frrtrace(9, frr_libfrr, event_cancel_async, master, NULL, NULL,
0, NULL, 0, 0, eventobj, 0);
assert(master->owner != pthread_self());
@ -1638,10 +1638,10 @@ static void thread_process_io(struct thread_master *m, unsigned int num)
ready++;
/*
* Unless someone has called thread_cancel from another
* Unless someone has called event_cancel from another
* pthread, the only thing that could have changed in
* m->handler.pfds while we were asleep is the .events
* field in a given pollfd. Barring thread_cancel() that
* field in a given pollfd. Barring event_cancel() that
* value should be a superset of the values we have in our
* copy, so there's no need to update it. Similarily,
* barring deletion, the fd should still be a valid index
@ -1758,7 +1758,7 @@ struct event *thread_fetch(struct thread_master *m, struct event *fetch)
pthread_mutex_lock(&m->mtx);
/* Process any pending cancellation requests */
do_thread_cancel(m);
do_event_cancel(m);
/*
* Attempt to flush ready queue before going into poll().

View File

@ -158,10 +158,10 @@ struct cpu_thread_history {
/*
* Please consider this macro deprecated, and do not use it in new code.
*/
#define THREAD_OFF(thread) \
do { \
if ((thread)) \
thread_cancel(&(thread)); \
#define THREAD_OFF(thread) \
do { \
if ((thread)) \
event_cancel(&(thread)); \
} while (0)
/*
@ -241,13 +241,12 @@ extern void _thread_execute(const struct xref_threadsched *xref,
struct thread_master *master,
void (*fn)(struct event *), void *arg, int val);
extern void thread_cancel(struct event **event);
extern void thread_cancel_async(struct thread_master *, struct event **,
void *);
extern void event_cancel(struct event **event);
extern void event_cancel_async(struct thread_master *, struct event **, void *);
/* Cancel ready tasks with an arg matching 'arg' */
extern void thread_cancel_event_ready(struct thread_master *m, void *arg);
extern void event_cancel_event_ready(struct thread_master *m, void *arg);
/* Cancel all tasks with an arg matching 'arg', including timers and io */
extern void thread_cancel_event(struct thread_master *m, void *arg);
extern void event_cancel_event(struct thread_master *m, void *arg);
extern struct event *thread_fetch(struct thread_master *, struct event *event);
extern void thread_call(struct event *event);
extern unsigned long thread_timer_remain_second(struct event *event);

View File

@ -190,7 +190,7 @@ int _frrzmq_event_add_read(const struct xref_threadsched *xref,
cb->in_cb = false;
if (events & ZMQ_POLLIN) {
thread_cancel(&cb->read.thread);
event_cancel(&cb->read.thread);
event_add_event(master, frrzmq_read_msg, cbp, fd,
&cb->read.thread);
@ -296,7 +296,7 @@ int _frrzmq_event_add_write(const struct xref_threadsched *xref,
cb->in_cb = false;
if (events & ZMQ_POLLOUT) {
thread_cancel(&cb->write.thread);
event_cancel(&cb->write.thread);
_event_add_event(xref, master, frrzmq_write_msg, cbp, fd,
&cb->write.thread);
@ -311,7 +311,7 @@ void frrzmq_thread_cancel(struct frrzmq_cb **cb, struct cb_core *core)
if (!cb || !*cb)
return;
core->cancelled = true;
thread_cancel(&core->thread);
event_cancel(&core->thread);
/* If cancelled from within a callback, don't try to free memory
* in this path.
@ -344,7 +344,7 @@ void frrzmq_check_events(struct frrzmq_cb **cbp, struct cb_core *core,
if ((events & event) && core->thread && !core->cancelled) {
struct thread_master *tm = core->thread->master;
thread_cancel(&core->thread);
event_cancel(&core->thread);
if (event == ZMQ_POLLIN)
event_add_event(tm, frrzmq_read_msg, cbp, cb->fd,

View File

@ -103,8 +103,8 @@ THREAD_OPERATION_TRACEPOINT_INSTANCE(schedule_timer)
THREAD_OPERATION_TRACEPOINT_INSTANCE(schedule_event)
THREAD_OPERATION_TRACEPOINT_INSTANCE(schedule_read)
THREAD_OPERATION_TRACEPOINT_INSTANCE(schedule_write)
THREAD_OPERATION_TRACEPOINT_INSTANCE(thread_cancel)
THREAD_OPERATION_TRACEPOINT_INSTANCE(thread_cancel_async)
THREAD_OPERATION_TRACEPOINT_INSTANCE(event_cancel)
THREAD_OPERATION_TRACEPOINT_INSTANCE(event_cancel_async)
THREAD_OPERATION_TRACEPOINT_INSTANCE(thread_call)
TRACEPOINT_EVENT(

View File

@ -265,7 +265,7 @@ int nb_cli_rpc(struct vty *vty, const char *xpath, struct list *input,
void nb_cli_confirmed_commit_clean(struct vty *vty)
{
thread_cancel(&vty->t_confirmed_commit_timeout);
event_cancel(&vty->t_confirmed_commit_timeout);
nb_config_free(vty->confirmed_commit_rollback);
vty->confirmed_commit_rollback = NULL;
}
@ -328,7 +328,7 @@ static int nb_cli_commit(struct vty *vty, bool force,
"%% Resetting confirmed-commit timeout to %u minute(s)\n\n",
confirmed_timeout);
thread_cancel(&vty->t_confirmed_commit_timeout);
event_cancel(&vty->t_confirmed_commit_timeout);
event_add_timer(master, nb_cli_confirmed_commit_timeout,
vty, confirmed_timeout * 60,
&vty->t_confirmed_commit_timeout);

View File

@ -97,8 +97,8 @@ void spf_backoff_free(struct spf_backoff *backoff)
if (!backoff)
return;
thread_cancel(&backoff->t_holddown);
thread_cancel(&backoff->t_timetolearn);
event_cancel(&backoff->t_holddown);
event_cancel(&backoff->t_timetolearn);
XFREE(MTYPE_SPF_BACKOFF_NAME, backoff->name);
XFREE(MTYPE_SPF_BACKOFF, backoff);
@ -150,7 +150,7 @@ long spf_backoff_schedule(struct spf_backoff *backoff)
break;
case SPF_BACKOFF_SHORT_WAIT:
case SPF_BACKOFF_LONG_WAIT:
thread_cancel(&backoff->t_holddown);
event_cancel(&backoff->t_holddown);
event_add_timer_msec(backoff->m, spf_backoff_holddown_elapsed,
backoff, backoff->holddown,
&backoff->t_holddown);

View File

@ -1053,7 +1053,7 @@ bool zlog_5424_apply_dst(struct zlog_cfg_5424 *zcf)
{
int fd = -1;
thread_cancel(&zcf->t_reconnect);
event_cancel(&zcf->t_reconnect);
if (zcf->prio_min != ZLOG_DISABLED)
fd = zlog_5424_open(zcf, -1);
@ -1106,7 +1106,7 @@ bool zlog_5424_rotate(struct zlog_cfg_5424 *zcf)
if (!zcf->active)
return true;
thread_cancel(&zcf->t_reconnect);
event_cancel(&zcf->t_reconnect);
/* need to retain the socket type because it also influences
* other fields (packets) and we can't atomically swap these

View File

@ -125,7 +125,7 @@ static void netlink_log_recv(struct event *t)
void netlink_set_nflog_group(int nlgroup)
{
if (netlink_log_fd >= 0) {
thread_cancel(&netlink_log_thread);
event_cancel(&netlink_log_thread);
close(netlink_log_fd);
netlink_log_fd = -1;
}

View File

@ -731,7 +731,7 @@ void interface_up(struct event *thread)
if (!oi->type_cfg)
oi->type = ospf6_default_iftype(oi->interface);
thread_cancel(&oi->thread_sso);
event_cancel(&oi->thread_sso);
if (IS_OSPF6_DEBUG_INTERFACE)
zlog_debug("Interface Event %s: [InterfaceUp]",
@ -935,7 +935,7 @@ void interface_down(struct event *thread)
if (oi->on_write_q) {
listnode_delete(ospf6->oi_write_q, oi);
if (list_isempty(ospf6->oi_write_q))
thread_cancel(&ospf6->t_write);
event_cancel(&ospf6->t_write);
oi->on_write_q = 0;
}

View File

@ -176,7 +176,7 @@ static int ospf6_vrf_disable(struct vrf *vrf)
* from VRF and make it "down".
*/
ospf6_vrf_unlink(ospf6, vrf);
thread_cancel(&ospf6->t_ospf6_receive);
event_cancel(&ospf6->t_ospf6_receive);
close(ospf6->fd);
ospf6->fd = -1;
}

View File

@ -356,7 +356,7 @@ void ospf_if_free(struct ospf_interface *oi)
listnode_delete(oi->ospf->oiflist, oi);
listnode_delete(oi->area->oiflist, oi);
thread_cancel_event(master, oi);
event_cancel_event(master, oi);
memset(oi, 0, sizeof(*oi));
XFREE(MTYPE_OSPF_IF, oi);

View File

@ -131,7 +131,7 @@ void ospf_nbr_free(struct ospf_neighbor *nbr)
THREAD_OFF(nbr->t_ls_upd);
/* Cancel all events. */ /* Thread lookup cost would be negligible. */
thread_cancel_event(master, nbr);
event_cancel_event(master, nbr);
bfd_sess_free(&nbr->bfd_session);

View File

@ -398,9 +398,9 @@ void pcep_thread_cancel_timer(struct event **thread)
}
if ((*thread)->master->owner == pthread_self()) {
thread_cancel(thread);
event_cancel(thread);
} else {
thread_cancel_async((*thread)->master, thread, NULL);
event_cancel_async((*thread)->master, thread, NULL);
}
}

View File

@ -190,17 +190,17 @@ void pcep_pcc_finalize(struct ctrl_state *ctrl_state,
}
if (pcc_state->t_reconnect != NULL) {
thread_cancel(&pcc_state->t_reconnect);
event_cancel(&pcc_state->t_reconnect);
pcc_state->t_reconnect = NULL;
}
if (pcc_state->t_update_best != NULL) {
thread_cancel(&pcc_state->t_update_best);
event_cancel(&pcc_state->t_update_best);
pcc_state->t_update_best = NULL;
}
if (pcc_state->t_session_timeout != NULL) {
thread_cancel(&pcc_state->t_session_timeout);
event_cancel(&pcc_state->t_session_timeout);
pcc_state->t_session_timeout = NULL;
}
@ -340,7 +340,7 @@ int pcep_pcc_enable(struct ctrl_state *ctrl_state, struct pcc_state *pcc_state)
assert(pcc_state->sess == NULL);
if (pcc_state->t_reconnect != NULL) {
thread_cancel(&pcc_state->t_reconnect);
event_cancel(&pcc_state->t_reconnect);
pcc_state->t_reconnect = NULL;
}
@ -408,7 +408,7 @@ int pcep_pcc_enable(struct ctrl_state *ctrl_state, struct pcc_state *pcc_state)
// In case some best pce alternative were waiting to activate
if (pcc_state->t_update_best != NULL) {
thread_cancel(&pcc_state->t_update_best);
event_cancel(&pcc_state->t_update_best);
pcc_state->t_update_best = NULL;
}

View File

@ -657,7 +657,7 @@ void path_ted_timer_handler_refresh(struct event *thread)
void path_ted_timer_sync_cancel(void)
{
if (ted_state_g.t_link_state_sync != NULL) {
thread_cancel(&ted_state_g.t_link_state_sync);
event_cancel(&ted_state_g.t_link_state_sync);
ted_state_g.t_link_state_sync = NULL;
}
}
@ -672,7 +672,7 @@ void path_ted_timer_sync_cancel(void)
void path_ted_timer_refresh_cancel(void)
{
if (ted_state_g.t_segment_list_refresh != NULL) {
thread_cancel(&ted_state_g.t_segment_list_refresh);
event_cancel(&ted_state_g.t_segment_list_refresh);
ted_state_g.t_segment_list_refresh = NULL;
}
}

View File

@ -1329,7 +1329,7 @@ void trigger_pathd_candidate_removed(struct srte_candidate *candidate)
/* The hook needs to be call synchronously, otherwise the candidate
path will be already deleted when the handler is called */
if (candidate->hook_timer != NULL) {
thread_cancel(&candidate->hook_timer);
event_cancel(&candidate->hook_timer);
candidate->hook_timer = NULL;
}
hook_call(pathd_candidate_removed, candidate);

View File

@ -211,8 +211,8 @@ struct pim_msdp {
event_add_write(mp->pim->msdp.master, pim_msdp_write, mp, mp->fd, \
&mp->t_write)
#define PIM_MSDP_PEER_READ_OFF(mp) thread_cancel(&mp->t_read)
#define PIM_MSDP_PEER_WRITE_OFF(mp) thread_cancel(&mp->t_write)
#define PIM_MSDP_PEER_READ_OFF(mp) event_cancel(&mp->t_read)
#define PIM_MSDP_PEER_WRITE_OFF(mp) event_cancel(&mp->t_write)
#if PIM_IPV != 6
// struct pim_msdp *msdp;

View File

@ -137,7 +137,7 @@ int main(int argc, char **argv)
continue;
XFREE(MTYPE_TMP, timers[index]->arg);
thread_cancel(&timers[index]);
event_cancel(&timers[index]);
timers_pending--;
}

View File

@ -44,7 +44,7 @@ int main(int argc, char **argv)
event_add_timer_msec(master, dummy_func, NULL, 0, &timers[i]);
}
for (i = 0; i < SCHEDULE_TIMERS; i++)
thread_cancel(&timers[i]);
event_cancel(&timers[i]);
monotime(&tv_start);
@ -62,7 +62,7 @@ int main(int argc, char **argv)
int index;
index = prng_rand(prng) % SCHEDULE_TIMERS;
thread_cancel(&timers[index]);
event_cancel(&timers[index]);
}
monotime(&tv_stop);

View File

@ -415,7 +415,7 @@ static void sigchild(void)
what = restart->what;
restart->pid = 0;
gs.numpids--;
thread_cancel(&restart->t_kill);
event_cancel(&restart->t_kill);
/* Update restart time to reflect the time the command
* completed. */
@ -688,7 +688,7 @@ static void handle_read(struct event *t_read)
dmn->name, (long)delay.tv_sec, (long)delay.tv_usec);
SET_READ_HANDLER(dmn);
thread_cancel(&dmn->t_wakeup);
event_cancel(&dmn->t_wakeup);
SET_WAKEUP_ECHO(dmn);
}
@ -874,7 +874,7 @@ static void phase_hanging(struct event *t_hanging)
static void set_phase(enum restart_phase new_phase)
{
gs.phase = new_phase;
thread_cancel(&gs.t_phase_hanging);
event_cancel(&gs.t_phase_hanging);
event_add_timer(master, phase_hanging, NULL, PHASE_TIMEOUT,
&gs.t_phase_hanging);

View File

@ -414,14 +414,14 @@ static void fpm_connect(struct event *t);
static void fpm_reconnect(struct fpm_nl_ctx *fnc)
{
/* Cancel all zebra threads first. */
thread_cancel_async(zrouter.master, &fnc->t_lspreset, NULL);
thread_cancel_async(zrouter.master, &fnc->t_lspwalk, NULL);
thread_cancel_async(zrouter.master, &fnc->t_nhgreset, NULL);
thread_cancel_async(zrouter.master, &fnc->t_nhgwalk, NULL);
thread_cancel_async(zrouter.master, &fnc->t_ribreset, NULL);
thread_cancel_async(zrouter.master, &fnc->t_ribwalk, NULL);
thread_cancel_async(zrouter.master, &fnc->t_rmacreset, NULL);
thread_cancel_async(zrouter.master, &fnc->t_rmacwalk, NULL);
event_cancel_async(zrouter.master, &fnc->t_lspreset, NULL);
event_cancel_async(zrouter.master, &fnc->t_lspwalk, NULL);
event_cancel_async(zrouter.master, &fnc->t_nhgreset, NULL);
event_cancel_async(zrouter.master, &fnc->t_nhgwalk, NULL);
event_cancel_async(zrouter.master, &fnc->t_ribreset, NULL);
event_cancel_async(zrouter.master, &fnc->t_ribwalk, NULL);
event_cancel_async(zrouter.master, &fnc->t_rmacreset, NULL);
event_cancel_async(zrouter.master, &fnc->t_rmacwalk, NULL);
/*
* Grab the lock to empty the streams (data plane might try to
@ -1484,9 +1484,9 @@ static int fpm_nl_finish_early(struct fpm_nl_ctx *fnc)
THREAD_OFF(fnc->t_rmacwalk);
THREAD_OFF(fnc->t_event);
THREAD_OFF(fnc->t_nhg);
thread_cancel_async(fnc->fthread->master, &fnc->t_read, NULL);
thread_cancel_async(fnc->fthread->master, &fnc->t_write, NULL);
thread_cancel_async(fnc->fthread->master, &fnc->t_connect, NULL);
event_cancel_async(fnc->fthread->master, &fnc->t_read, NULL);
event_cancel_async(fnc->fthread->master, &fnc->t_write, NULL);
event_cancel_async(fnc->fthread->master, &fnc->t_connect, NULL);
if (fnc->socket != -1) {
close(fnc->socket);

View File

@ -5874,11 +5874,11 @@ void zebra_dplane_ns_enable(struct zebra_ns *zns, bool enabled)
/* Stop any outstanding tasks */
if (zdplane_info.dg_master) {
thread_cancel_async(zdplane_info.dg_master,
&zi->t_request, NULL);
event_cancel_async(zdplane_info.dg_master,
&zi->t_request, NULL);
thread_cancel_async(zdplane_info.dg_master, &zi->t_read,
NULL);
event_cancel_async(zdplane_info.dg_master, &zi->t_read,
NULL);
}
XFREE(MTYPE_DP_NS, zi);
@ -6854,8 +6854,8 @@ void zebra_dplane_shutdown(void)
zdplane_info.dg_run = false;
if (zdplane_info.dg_t_update)
thread_cancel_async(zdplane_info.dg_t_update->master,
&zdplane_info.dg_t_update, NULL);
event_cancel_async(zdplane_info.dg_t_update->master,
&zdplane_info.dg_t_update, NULL);
frr_pthread_stop(zdplane_info.dg_pthread, NULL);

View File

@ -670,7 +670,7 @@ void zserv_close_client(struct zserv *client)
zlog_debug("Closing client '%s'",
zebra_route_string(client->proto));
thread_cancel_event(zrouter.master, client);
event_cancel_event(zrouter.master, client);
THREAD_OFF(client->t_cleanup);
THREAD_OFF(client->t_process);