sched/pthread/join: refactor pthread join to support join task

1. add support to join main task

| static pthread_t self;
|
| static void *join_task(void *arg)
| {
|   int ret;
|   ret = pthread_join(self, NULL);          <---  /* Fix Task could not be joined */
|   return NULL;
| }
|
| int main(int argc, char *argv[])
| {
|   pthread_t thread;
|
|   self = pthread_self();
|
|   pthread_create(&thread, NULL, join_task, NULL);
|   sleep(1);
|
|   pthread_exit(NULL);
|   return 0;
| }

2. Detach active thread will not alloc for additional join, just update the task flag.
3. Remove the return value waiting lock logic (data_sem),
   the return value will be stored in the waiting tcb.
4. Revise the return value of pthread_join(), consistent with linux
   e.g:
       Joining a detached and canceled thread should return EINVAL, not ESRCH

   https://pubs.opengroup.org/onlinepubs/009695399/functions/pthread_join.html

   [EINVAL]
      The value specified by thread does not refer to a joinable thread.

NOTE:

This PR will not increase stack usage, but struct tcb_s will increase 32 bytes.

Signed-off-by: chao an <anchao@lixiang.com>
This commit is contained in:
chao an 2024-03-12 15:10:11 +08:00 committed by Masayuki Ishikawa
parent 1fa2559f00
commit df30a1f8d3
17 changed files with 323 additions and 493 deletions

View File

@ -104,7 +104,8 @@
#define TCB_FLAG_HEAP_DUMP (1 << 11) /* Bit 11: Heap dump */
#define TCB_FLAG_DETACHED (1 << 12) /* Bit 12: Pthread detached */
#define TCB_FLAG_FORCED_CANCEL (1 << 13) /* Bit 13: Pthread cancel is forced */
#define TCB_FLAG_FREE_TCB (1 << 14) /* Bit 14: Free tcb after exit */
#define TCB_FLAG_JOIN_COMPLETED (1 << 14) /* Bit 14: Pthread join completed */
#define TCB_FLAG_FREE_TCB (1 << 15) /* Bit 15: Free tcb after exit */
/* Values for struct task_group tg_flags */
@ -378,6 +379,18 @@ struct stackinfo_s
/* from the stack. */
};
/* struct task_join_s *******************************************************/
/* Used to save task join information */
struct task_join_s
{
sq_entry_t entry; /* Implements link list */
pid_t pid; /* Includes pid */
bool detached; /* true: pthread_detached'ed */
pthread_addr_t exit_value; /* Returned data */
};
/* struct task_group_s ******************************************************/
/* All threads created by pthread_create belong in the same task group (along
@ -388,7 +401,6 @@ struct stackinfo_s
* This structure should contain *all* resources shared by tasks and threads
* that belong to the same task group:
*
* Child exit status
* Environment variables
* PIC data space and address environments
* File descriptors
@ -403,10 +415,6 @@ struct stackinfo_s
* the struct task_group_s is free.
*/
#ifndef CONFIG_DISABLE_PTHREAD
struct join_s; /* Forward reference */
/* Defined in sched/pthread/pthread.h */
#endif
#ifdef CONFIG_BINFMT_LOADABLE
struct binary_s; /* Forward reference */
/* Defined in include/nuttx/binfmt/binfmt.h */
@ -466,11 +474,8 @@ struct task_group_s
#ifndef CONFIG_DISABLE_PTHREAD
/* Pthreads ***************************************************************/
/* Pthread join Info: */
mutex_t tg_joinlock; /* Mutually exclusive access to join data */
FAR struct join_s *tg_joinhead; /* Head of a list of join data */
FAR struct join_s *tg_jointail; /* Tail of a list of join data */
rmutex_t tg_joinlock; /* Mutually exclusive access to join queue */
sq_queue_t tg_joinqueue; /* List of join status of tcb */
#endif
/* Thread local storage ***************************************************/
@ -542,6 +547,15 @@ struct tcb_s
sq_entry_t member; /* List entry of task member */
#endif
/* Task join **************************************************************/
#ifndef CONFIG_DISABLE_PTHREAD
sq_queue_t join_queue; /* List of wait entries for task */
sq_entry_t join_entry; /* List entry of task join */
sem_t join_sem; /* Semaphore for task join */
pthread_addr_t join_val; /* Returned data */
#endif
/* Address Environment ****************************************************/
#ifdef CONFIG_ARCH_ADDRENV
@ -570,7 +584,7 @@ struct tcb_s
uint8_t cpu; /* CPU index if running/assigned */
cpu_set_t affinity; /* Bit set of permitted CPUs */
#endif
uint16_t flags; /* Misc. general status flags */
uint32_t flags; /* Misc. general status flags */
int16_t lockcount; /* 0=preemptible (not-locked) */
#ifdef CONFIG_IRQCOUNT
int16_t irqcount; /* 0=Not in critical section */
@ -714,7 +728,6 @@ struct pthread_tcb_s
pthread_trampoline_t trampoline; /* User-space pthread startup function */
pthread_addr_t arg; /* Startup argument */
bool join_complete; /* Join was completed */
};
#endif /* !CONFIG_DISABLE_PTHREAD */

View File

@ -161,9 +161,10 @@ int group_initialize(FAR struct task_tcb_s *tcb, uint8_t ttype)
}
#ifndef CONFIG_DISABLE_PTHREAD
/* Initialize the pthread join mutex */
/* Initialize the task group join */
nxmutex_init(&group->tg_joinlock);
nxrmutex_init(&group->tg_joinlock);
sq_init(&group->tg_joinqueue);
#endif
#if defined(CONFIG_SCHED_WAITPID) && !defined(CONFIG_SCHED_HAVE_PARENT)

View File

@ -46,6 +46,7 @@
#include <nuttx/drivers/drivers.h>
#include <nuttx/init.h>
#include "task/task.h"
#include "sched/sched.h"
#include "signal/signal.h"
#include "semaphore/semaphore.h"
@ -519,6 +520,10 @@ void nx_start(void)
DEBUGVERIFY(group_initialize(&g_idletcb[i], g_idletcb[i].cmn.flags));
g_idletcb[i].cmn.group->tg_info->ta_argv = &g_idleargv[i][0];
/* Initialize the task join */
nxtask_joininit(&g_idletcb[i].cmn);
#ifdef CONFIG_SMP
/* Create a stack for all CPU IDLE threads (except CPU0 which already
* has a stack).

View File

@ -36,29 +36,6 @@
#include <nuttx/semaphore.h>
#include <nuttx/sched.h>
/****************************************************************************
* Public Type Declarations
****************************************************************************/
/* The following defines an entry in the pthread logic's local data set.
* Note that this structure is used to implemented a singly linked list.
* This structure is used (instead of, say, a binary search tree) because
* the data set will be searched using the pid as a key -- a process IDs will
* always be created in a montonically increasing fashion.
*/
struct join_s
{
FAR struct join_s *next; /* Implements link list */
uint8_t crefs; /* Reference count */
bool detached; /* true: pthread_detached'ed */
bool terminated; /* true: detach'ed+exit'ed */
pthread_t thread; /* Includes pid */
sem_t exit_sem; /* Implements join */
sem_t data_sem; /* Implements join */
pthread_addr_t exit_value; /* Returned data */
};
/****************************************************************************
* Public Data
****************************************************************************/
@ -83,9 +60,9 @@ int pthread_setup_scheduler(FAR struct pthread_tcb_s *tcb, int priority,
int pthread_completejoin(pid_t pid, FAR void *exit_value);
void pthread_destroyjoin(FAR struct task_group_s *group,
FAR struct join_s *pjoin);
int pthread_findjoininfo(FAR struct task_group_s *group,
pid_t pid, FAR struct join_s **join);
FAR struct task_join_s *pjoin);
int pthread_findjoininfo(FAR struct task_group_s *group, pid_t pid,
FAR struct task_join_s **join, bool create);
void pthread_release(FAR struct task_group_s *group);
int pthread_sem_take(FAR sem_t *sem, FAR const struct timespec *abs_timeout);

View File

@ -24,6 +24,7 @@
#include <nuttx/config.h>
#include <nuttx/nuttx.h>
#include <sys/types.h>
#include <stdbool.h>
#include <pthread.h>
@ -35,133 +36,6 @@
#include "group/group.h"
#include "pthread/pthread.h"
/****************************************************************************
* Private Functions
****************************************************************************/
/****************************************************************************
* Name: pthread_notifywaiters
*
* Description:
* Notify all other threads waiting in phread join for this thread's
* exit data. This must be done by the child at child thread
* destruction time.
*
****************************************************************************/
static bool pthread_notifywaiters(FAR struct join_s *pjoin)
{
int ntasks_waiting;
int status;
sinfo("pjoin=%p\n", pjoin);
/* Are any tasks waiting for our exit value? */
status = nxsem_get_value(&pjoin->exit_sem, &ntasks_waiting);
if (status == OK && ntasks_waiting < 0)
{
/* Set the data semaphore so that this thread will be
* awakened when all waiting tasks receive the data
*/
nxsem_init(&pjoin->data_sem, 0, (ntasks_waiting + 1));
/* Post the semaphore to restart each thread that is waiting
* on the semaphore
*/
do
{
status = nxsem_post(&pjoin->exit_sem);
if (status == OK)
{
status = nxsem_get_value(&pjoin->exit_sem, &ntasks_waiting);
}
}
while (ntasks_waiting < 0 && status == OK);
/* Now wait for all these restarted tasks to obtain the return
* value.
*/
nxsem_wait_uninterruptible(&pjoin->data_sem);
return true;
}
return false;
}
/****************************************************************************
* Name: pthread_removejoininfo
*
* Description:
* Remove a join structure from the local data set.
*
* Input Parameters:
* pid
*
* Returned Value:
* None.
*
* Assumptions:
* The caller has provided protection from re-entrancy.
*
****************************************************************************/
static void pthread_removejoininfo(FAR struct task_group_s *group,
pid_t pid)
{
FAR struct join_s *prev;
FAR struct join_s *join;
/* Find the entry with the matching pid */
for (prev = NULL, join = group->tg_joinhead;
(join && (pid_t)join->thread != pid);
prev = join, join = join->next);
/* Remove it from the data set. */
/* First check if this is the entry at the head of the list. */
if (join)
{
if (!prev)
{
/* Check if this is the only entry in the list */
if (!join->next)
{
group->tg_joinhead = NULL;
group->tg_jointail = NULL;
}
/* Otherwise, remove it from the head of the list */
else
{
group->tg_joinhead = join->next;
}
}
/* It is not at the head of the list, check if it is at the tail. */
else if (!join->next)
{
group->tg_jointail = prev;
prev->next = NULL;
}
/* No, remove it from the middle of the list. */
else
{
prev->next = join->next;
}
}
}
/****************************************************************************
* Public Functions
****************************************************************************/
@ -189,59 +63,53 @@ static void pthread_removejoininfo(FAR struct task_group_s *group,
int pthread_completejoin(pid_t pid, FAR void *exit_value)
{
FAR struct tcb_s *tcb = nxsched_get_tcb(pid);
FAR struct task_group_s *group = tcb ? tcb->group : NULL;
FAR struct join_s *pjoin;
int ret;
FAR struct task_group_s *group = tcb->group;
FAR struct task_join_s *join;
FAR struct tcb_s *wtcb;
FAR sq_entry_t *curr;
FAR sq_entry_t *next;
int ret = OK;
sinfo("pid=%d exit_value=%p group=%p\n", pid, exit_value, group);
DEBUGASSERT(group && tcb);
sinfo("pid=%d exit_value=%p\n", pid, exit_value);
/* First, find thread's structure in the private data set. */
nxrmutex_lock(&group->tg_joinlock);
nxmutex_lock(&group->tg_joinlock);
ret = pthread_findjoininfo(group, pid, &pjoin);
if (ret != OK)
if (!sq_empty(&tcb->join_queue))
{
nxmutex_unlock(&group->tg_joinlock);
return ((tcb->flags & TCB_FLAG_DETACHED) ||
(tcb->flags & TCB_FLAG_TTYPE_MASK) != TCB_FLAG_TTYPE_PTHREAD) ?
OK : ERROR;
}
else
{
FAR struct pthread_tcb_s *ptcb = (FAR struct pthread_tcb_s *)tcb;
bool waiters;
/* Save the return exit value in the thread structure. */
pjoin->terminated = true;
pjoin->exit_value = exit_value;
ptcb->join_complete = true;
/* Notify waiters of the availability of the exit value */
waiters = pthread_notifywaiters(pjoin);
/* If there are no waiters and if the thread is marked as detached.
* then discard the join information now. Otherwise, the pthread
* join logic will call pthread_destroyjoin() when all of the threads
* have sampled the exit value.
*/
if (!waiters && pjoin->detached)
sq_for_every_safe(&tcb->join_queue, curr, next)
{
pthread_destroyjoin(group, pjoin);
/* Remove join entry from queue */
sq_rem(curr, &tcb->join_queue);
/* Get tcb entry which waiting for the join */
wtcb = container_of(curr, struct tcb_s, join_entry);
/* Save the return exit value in the thread structure. */
wtcb->join_val = exit_value;
/* Notify waiters of the availability of the exit value */
nxsem_post(&wtcb->join_sem);
}
}
else if (!sq_is_singular(&tcb->group->tg_members))
{
ret = pthread_findjoininfo(tcb->group, pid, &join, true);
if (ret == OK)
{
join->detached = !!(tcb->flags & TCB_FLAG_DETACHED);
join->exit_value = exit_value;
}
/* Giving the following semaphore will allow the waiters
* to call pthread_destroyjoin.
*/
nxmutex_unlock(&group->tg_joinlock);
}
return OK;
tcb->flags |= TCB_FLAG_JOIN_COMPLETED;
nxrmutex_unlock(&group->tg_joinlock);
return ret;
}
/****************************************************************************
@ -261,18 +129,15 @@ int pthread_completejoin(pid_t pid, FAR void *exit_value)
****************************************************************************/
void pthread_destroyjoin(FAR struct task_group_s *group,
FAR struct join_s *pjoin)
FAR struct task_join_s *pjoin)
{
sinfo("pjoin=%p\n", pjoin);
/* Remove the join info from the set of joins */
pthread_removejoininfo(group, (pid_t)pjoin->thread);
/* Destroy its semaphores */
nxsem_destroy(&pjoin->data_sem);
nxsem_destroy(&pjoin->exit_sem);
nxrmutex_lock(&group->tg_joinlock);
sq_rem(&pjoin->entry, &group->tg_joinqueue);
nxrmutex_unlock(&group->tg_joinlock);
/* And deallocate the pjoin structure */

View File

@ -40,6 +40,7 @@
#include <nuttx/kmalloc.h>
#include <nuttx/pthread.h>
#include "task/task.h"
#include "sched/sched.h"
#include "group/group.h"
#include "clock/clock.h"
@ -217,6 +218,10 @@ int nx_pthread_create(pthread_trampoline_t trampoline, FAR pthread_t *thread,
ptcb->cmn.flags |= TCB_FLAG_FREE_TCB;
/* Initialize the task join */
nxtask_joininit(&ptcb->cmn);
/* Bind the parent's group to the new TCB (we have not yet joined the
* group).
*/

View File

@ -64,45 +64,45 @@ int pthread_detach(pthread_t thread)
{
FAR struct tcb_s *rtcb = this_task();
FAR struct task_group_s *group = rtcb->group;
FAR struct join_s *pjoin;
FAR struct task_join_s *join;
FAR struct tcb_s *tcb;
int ret;
sinfo("Thread=%d group=%p\n", thread, group);
DEBUGASSERT(group);
nxrmutex_lock(&group->tg_joinlock);
/* Find the entry associated with this pthread. */
nxmutex_lock(&group->tg_joinlock);
ret = pthread_findjoininfo(group, (pid_t)thread, &pjoin);
if (ret == OK)
tcb = nxsched_get_tcb((pid_t)thread);
if (tcb == NULL)
{
/* Has the thread already terminated? */
/* If tcb has been destroyed, update the pending join
* status in the group.
*/
if (pjoin->terminated)
ret = pthread_findjoininfo(group, (pid_t)thread, &join, false);
if (ret == OK)
{
/* YES.. just remove the thread entry. */
pthread_destroyjoin(group, pjoin);
join->detached = true;
}
else
{
/* NO.. Just mark the thread as detached. It
* will be removed and deallocated when the
* thread exits
*/
if (pjoin->detached)
{
ret = EINVAL;
}
else
{
pjoin->detached = true;
}
ret = ESRCH;
}
goto errout;
}
nxmutex_unlock(&group->tg_joinlock);
if ((group != tcb->group) ||
(tcb->flags & TCB_FLAG_DETACHED) != 0)
{
ret = EINVAL;
}
else
{
tcb->flags |= TCB_FLAG_DETACHED;
ret = OK;
}
errout:
nxrmutex_unlock(&group->tg_joinlock);
sinfo("Returning %d\n", ret);
return ret;

View File

@ -28,6 +28,8 @@
#include <assert.h>
#include <debug.h>
#include <nuttx/nuttx.h>
#include "group/group.h"
#include "pthread/pthread.h"
@ -35,69 +37,6 @@
* Private Functions
****************************************************************************/
/****************************************************************************
* Name: pthread_createjoininfo
*
* Description:
* Allocate a detachable structure to support pthread_join logic and add
* the joininfo to the thread.
*
* Input Parameters:
* ptcb
*
* Output Parameters:
* pjoin - joininfo point
*
* Returned Value:
* 0 if successful.
*
* Assumptions:
*
****************************************************************************/
int pthread_createjoininfo(FAR struct pthread_tcb_s *ptcb,
FAR struct join_s **pjoin)
{
/* Allocate a detachable structure to support pthread_join logic */
*pjoin = kmm_zalloc(sizeof(struct join_s));
if (*pjoin == NULL)
{
serr("ERROR: Failed to allocate join\n");
return EINVAL;
}
(*pjoin)->thread = (pthread_t)ptcb->cmn.pid;
/* Initialize the semaphore in the join structure to zero. */
if (nxsem_init(&(*pjoin)->exit_sem, 0, 0) < 0)
{
kmm_free(*pjoin);
return EINVAL;
}
else
{
FAR struct task_group_s *group = ptcb->cmn.group;
/* Attach the join info to the TCB. */
(*pjoin)->next = NULL;
if (!group->tg_jointail)
{
group->tg_joinhead = *pjoin;
}
else
{
group->tg_jointail->next = *pjoin;
}
group->tg_jointail = *pjoin;
}
return OK;
}
/****************************************************************************
* Public Functions
****************************************************************************/
@ -128,55 +67,48 @@ int pthread_createjoininfo(FAR struct pthread_tcb_s *ptcb,
*
****************************************************************************/
int pthread_findjoininfo(FAR struct task_group_s *group,
pid_t pid, FAR struct join_s **pjoin)
int pthread_findjoininfo(FAR struct task_group_s *group, pid_t pid,
FAR struct task_join_s **pjoin, bool create)
{
FAR struct pthread_tcb_s *ptcb;
FAR struct tcb_s *tcb;
FAR struct task_join_s *join;
FAR sq_entry_t *curr;
FAR sq_entry_t *next;
DEBUGASSERT(group);
nxrmutex_lock(&group->tg_joinlock);
/* Find the entry with the matching pid */
for (*pjoin = group->tg_joinhead;
(*pjoin && (pid_t)(*pjoin)->thread != pid);
*pjoin = (*pjoin)->next);
/* and return it */
if (*pjoin)
sq_for_every_safe(&group->tg_joinqueue, curr, next)
{
return OK;
join = container_of(curr, struct task_join_s, entry);
if (join->pid == pid)
{
goto found;
}
}
/* Task has been deleted, return ESRCH */
nxrmutex_unlock(&group->tg_joinlock);
tcb = nxsched_get_tcb(pid);
if (tcb == NULL)
{
return ESRCH;
}
/* Task was detached or not a pthread, return EINVAL */
if ((tcb->flags & TCB_FLAG_DETACHED) != 0 ||
(tcb->flags & TCB_FLAG_TTYPE_MASK) != TCB_FLAG_TTYPE_PTHREAD)
if (!create)
{
return EINVAL;
}
ptcb = (FAR struct pthread_tcb_s *)tcb;
/* Task was join completed, is in the process
* of being deleted, return ESRCH
*/
if (ptcb->join_complete)
join = kmm_zalloc(sizeof(struct task_join_s));
if (join == NULL)
{
return ESRCH;
return ENOMEM;
}
/* Else create joininfo for the task */
join->pid = pid;
return pthread_createjoininfo(ptcb, pjoin);
nxrmutex_lock(&group->tg_joinlock);
sq_addfirst(&join->entry, &group->tg_joinqueue);
found:
nxrmutex_unlock(&group->tg_joinlock);
*pjoin = join;
return OK;
}

View File

@ -34,7 +34,6 @@
#include <nuttx/cancelpt.h>
#include "sched/sched.h"
#include "group/group.h"
#include "pthread/pthread.h"
/****************************************************************************
@ -74,153 +73,103 @@ int pthread_join(pthread_t thread, FAR pthread_addr_t *pexit_value)
{
FAR struct tcb_s *rtcb = this_task();
FAR struct task_group_s *group = rtcb->group;
FAR struct join_s *pjoin;
int ret;
sinfo("thread=%d group=%p\n", thread, group);
DEBUGASSERT(group);
FAR struct task_join_s *join;
FAR struct tcb_s *tcb;
int ret = OK;
/* pthread_join() is a cancellation point */
enter_cancellation_point();
/* First make sure that this is not an attempt to join to
* ourself.
*/
nxrmutex_lock(&group->tg_joinlock);
if ((pid_t)thread == nxsched_gettid())
tcb = nxsched_get_tcb((pid_t)thread);
if (tcb == NULL || (tcb->flags & TCB_FLAG_JOIN_COMPLETED) != 0)
{
leave_cancellation_point();
return EDEADLK;
}
/* Make sure no other task is mucking with the data structures
* while we are performing the following operations. NOTE:
* we can be also sure that pthread_exit() will not execute
* because it will also attempt to get this semaphore.
*/
nxmutex_lock(&group->tg_joinlock);
/* Find the join information associated with this thread.
* This can fail for one of three reasons: (1) There is no
* thread associated with 'thread,' (2) the thread is a task
* and does not have join information, or (3) the thread
* was detached and has exited.
*/
ret = pthread_findjoininfo(group, (pid_t)thread, &pjoin);
if (ret == OK)
{
if (pjoin->detached)
ret = pthread_findjoininfo(group, (pid_t)thread, &join, false);
if (ret == OK)
{
nxmutex_unlock(&group->tg_joinlock);
leave_cancellation_point();
return EINVAL;
}
/* If the task is terminated, maintain the same behavior as Linux:
* 1. Join detached task will always return EINVAL.
* 2. Other threads will destroy the join information after
* obtain the exit value, ESRCH will return if calling
* pthread_join() again
*/
/* NOTE: sched_lock() is not enough for SMP
* because another CPU would continue the pthread and exit
* sequences so need to protect it with a critical section
*/
#ifdef CONFIG_SMP
irqstate_t flags = enter_critical_section();
#endif
/* We found the join info structure. Increment for the reference
* to the join structure that we have. This will keep things
* stable for we have to do
*/
sched_lock();
pjoin->crefs++;
/* Check if the thread is still running. If not, then things are
* simpler. There are still race conditions to be concerned with.
* For example, there could be multiple threads executing in the
* 'else' block below when we enter!
*/
if (pjoin->terminated)
{
sinfo("Thread has terminated\n");
/* Get the thread exit value from the terminated thread. */
if (pexit_value)
if (join->detached)
{
sinfo("exit_value=%p\n", pjoin->exit_value);
*pexit_value = pjoin->exit_value;
ret = EINVAL;
}
else
{
if (pexit_value != NULL)
{
*pexit_value = join->exit_value;
}
pthread_destroyjoin(group, join);
}
}
else
{
sinfo("Thread is still running\n");
/* Relinquish the data set semaphore. Since pre-emption is
* disabled, we can be certain that no task has the
* opportunity to run between the time we relinquish the
* join semaphore and the time that we wait on the thread exit
* semaphore.
*/
nxmutex_unlock(&group->tg_joinlock);
/* Take the thread's thread exit semaphore. We will sleep here
* until the thread exits. We need to exercise caution because
* there could be multiple threads waiting here for the same
* pthread to exit.
*/
nxsem_wait_uninterruptible(&pjoin->exit_sem);
/* The thread has exited! Get the thread exit value */
if (pexit_value)
{
*pexit_value = pjoin->exit_value;
sinfo("exit_value=%p\n", pjoin->exit_value);
}
/* Post the thread's data semaphore so that the exiting thread
* will know that we have received the data.
*/
nxsem_post(&pjoin->data_sem);
/* Retake the join semaphore, we need to hold this when
* pthread_destroyjoin is called.
*/
nxmutex_lock(&group->tg_joinlock);
ret = ESRCH;
}
/* Pre-emption is okay now. The logic still cannot be re-entered
* because we hold the join semaphore
*/
sched_unlock();
#ifdef CONFIG_SMP
leave_critical_section(flags);
#endif
/* Release our reference to the join structure and, if the reference
* count decrements to zero, deallocate the join structure.
*/
if (--pjoin->crefs <= 0)
{
pthread_destroyjoin(group, pjoin);
}
ret = OK;
goto errout;
}
nxmutex_unlock(&group->tg_joinlock);
/* First make sure that this is not an attempt to join to
* ourself.
*/
if (tcb == rtcb)
{
ret = EDEADLK;
goto errout;
}
/* Task was detached or not a pthread, return EINVAL */
if ((tcb->group != group) ||
(tcb->flags & TCB_FLAG_DETACHED) != 0)
{
ret = EINVAL;
goto errout;
}
/* Relinquish the data set semaphore. Since pre-emption is
* disabled, we can be certain that no task has the
* opportunity to run between the time we relinquish the
* join semaphore and the time that we wait on the thread exit
* semaphore.
*/
sq_addfirst(&rtcb->join_entry, &tcb->join_queue);
nxrmutex_unlock(&group->tg_joinlock);
/* Take the thread's thread exit semaphore. We will sleep here
* until the thread exits. We need to exercise caution because
* there could be multiple threads waiting here for the same
* pthread to exit.
*/
nxsem_wait_uninterruptible(&rtcb->join_sem);
nxrmutex_lock(&group->tg_joinlock);
/* The thread has exited! Get the thread exit value */
if (pexit_value != NULL)
{
*pexit_value = rtcb->join_val;
}
errout:
nxrmutex_unlock(&group->tg_joinlock);
leave_cancellation_point();
sinfo("Returning %d\n", ret);
sinfo("Returning %d, exit_value %p\n", ret, *pexit_value);
return ret;
}

View File

@ -28,6 +28,7 @@
#include <assert.h>
#include <debug.h>
#include <nuttx/nuttx.h>
#include <nuttx/kmalloc.h>
#include "pthread/pthread.h"
@ -58,32 +59,22 @@
void pthread_release(FAR struct task_group_s *group)
{
FAR struct join_s *join;
DEBUGASSERT(group);
FAR sq_entry_t *curr;
FAR sq_entry_t *next;
/* Visit and delete each join structure still in the list. Since we
* are last exiting thread of the group, no special protection should
* be required.
*/
while (group->tg_joinhead)
sq_for_every_safe(&group->tg_joinqueue, curr, next)
{
/* Remove the join from the head of the list. */
/* Deallocate the join structure */
join = group->tg_joinhead;
group->tg_joinhead = join->next;
/* Destroy the join semaphores */
nxsem_destroy(&join->data_sem);
nxsem_destroy(&join->exit_sem);
/* And deallocate the join structure */
kmm_free(join);
kmm_free(container_of(curr, struct task_join_s, entry));
}
/* Destroy the join list mutex */
nxmutex_destroy(&group->tg_joinlock);
nxrmutex_destroy(&group->tg_joinlock);
}

View File

@ -31,6 +31,7 @@
#include <nuttx/arch.h>
#include <nuttx/sched.h>
#include "task/task.h"
#include "sched/sched.h"
#include "group/group.h"
#include "timer/timer.h"
@ -164,9 +165,13 @@ int nxsched_release_tcb(FAR struct tcb_s *tcb, uint8_t ttype)
group_leave(tcb);
#ifndef CONFIG_DISABLE_PTHREAD
/* Destroy the pthread join mutex */
nxtask_joindestroy(tcb);
/* Kernel thread and group still reference by pthread */
#ifndef CONFIG_DISABLE_PTHREAD
if (ttype != TCB_FLAG_TTYPE_PTHREAD)
{
ttcb = (FAR struct task_tcb_s *)tcb;

View File

@ -36,6 +36,7 @@ set(SRCS
task_cancelpt.c
task_terminate.c
task_gettid.c
task_join.c
exit.c)
if(CONFIG_SCHED_HAVE_PARENT)

View File

@ -22,7 +22,7 @@ CSRCS += task_create.c task_init.c task_setup.c task_activate.c
CSRCS += task_start.c task_delete.c task_exit.c task_exithook.c
CSRCS += task_getgroup.c task_getpid.c task_prctl.c task_recover.c
CSRCS += task_restart.c task_spawnparms.c task_cancelpt.c task_terminate.c
CSRCS += task_gettid.c exit.c
CSRCS += task_gettid.c exit.c task_join.c
ifeq ($(CONFIG_SCHED_HAVE_PARENT),y)
CSRCS += task_getppid.c task_reparent.c

View File

@ -58,4 +58,14 @@ void nxtask_recover(FAR struct tcb_s *tcb);
bool nxnotify_cancellation(FAR struct tcb_s *tcb);
/* Task Join */
#ifndef CONFIG_DISABLE_PTHREAD
void nxtask_joininit(FAR struct tcb_s *tcb);
void nxtask_joindestroy(FAR struct tcb_s *tcb);
#else
# define nxtask_joininit(tcb)
# define nxtask_joindestroy(tcb)
#endif
#endif /* __SCHED_TASK_TASK_H */

View File

@ -144,6 +144,10 @@ FAR struct task_tcb_s *nxtask_setup_fork(start_t retaddr)
child->cmn.flags |= TCB_FLAG_FREE_TCB;
/* Initialize the task join */
nxtask_joininit(&child->cmn);
/* Allocate a new task group with the same privileges as the parent */
ret = group_initialize(child, ttype);

View File

@ -118,6 +118,12 @@ int nxtask_init(FAR struct task_tcb_s *tcb, const char *name, int priority,
return ret;
}
#ifndef CONFIG_DISABLE_PTHREAD
/* Initialize the task join */
nxtask_joininit(&tcb->cmn);
#endif
/* Duplicate the parent tasks environment */
ret = env_dup(tcb->cmn.group, envp);
@ -204,6 +210,8 @@ errout_with_group:
}
}
nxtask_joindestroy(&tcb->cmn);
group_leave(&tcb->cmn);
sched_trace_end();

64
sched/task/task_join.c Normal file
View File

@ -0,0 +1,64 @@
/****************************************************************************
* sched/task/task_join.c
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. The
* ASF licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the
* License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*
****************************************************************************/
/****************************************************************************
* Included Files
****************************************************************************/
#include <nuttx/config.h>
#include <nuttx/nuttx.h>
#include <sys/types.h>
#include <stdbool.h>
#include <pthread.h>
#include <assert.h>
#include <errno.h>
#include <debug.h>
#include "sched/sched.h"
#include "group/group.h"
#include "pthread/pthread.h"
#ifndef CONFIG_DISABLE_PTHREAD
/****************************************************************************
* Public Functions
****************************************************************************/
/****************************************************************************
* Name: nxtask_joindestroy
****************************************************************************/
void nxtask_joindestroy(FAR struct tcb_s *tcb)
{
nxsem_destroy(&tcb->join_sem);
}
/****************************************************************************
* Name: nxtask_joininit
****************************************************************************/
void nxtask_joininit(FAR struct tcb_s *tcb)
{
sq_init(&tcb->join_queue);
nxsem_init(&tcb->join_sem, 0, 0);
}
#endif /* !CONFIG_DISABLE_PTHREAD */