Bladeren bron

[LibOS,Pal] Correctly emulate CLONE_CHILD_CLEARTID

When child thread exits, it wakes up its parent if CLONE_CHILD_CLEARTID was set
during clone() call. Previously, this was done by the child thread itself as
part of its own clean-up in release_clear_child_id(). But this child thread is
still alive at this point and uses some resources, most notably the stack (that
might have been provided by the parent) and the SGX TCS slot. Upon waking up,
the parent might decide to free that stack (as Pthreads do) or re-use the TCS
slot, causing data races.

This commit introduces a correct emulation of CLONE_CHILD_CLEARTID:
- A new argument `PAL_PTR clear_child_tid` is added to DkThreadExit();
  it points to internal Graphene memory that is erased on child exit to notify
  Async Helper thread.
- At PAL layer, when thread finally exits, it sets PAL-level *clear_child_tid = 0
  (corresponds to &clear_child_tid_val_pal at LibOS level);  this signals to LibOS
  layer that the thread stopped using resources.
- At LibOS layer, Async Helper thread is set up to wait for the signal from
  PAL; it is now the responsibility of Async Helper thread to call
  release_clear_child_id() to wake up the parent thread.
- Async Helper thread waits for clear_child_tid_val_pal == 0 and then sets
  the actual clear_child_tid to 0 and wakes up the waiting parent.

Note that for Linux-SGX PAL, clear_child_tid is set to 0 not immediately
but as part of handle_thread_reset, otherwise the TCS slot could be still
occupied when LibOS wakes up the parent.

As a side effect, the LibOS code for threads/process exit is cleaned up.

This commit also fixes all regression tests to use the new signature of
DkThreadExit() and increases the number of SGX threads slightly (to
accommodate the newly used Async Helper thread).
Dmitrii Kuvaiskii 4 jaren geleden
bovenliggende
commit
b194aa17fb
36 gewijzigde bestanden met toevoegingen van 253 en 110 verwijderingen
  1. 4 2
      Documentation/oldwiki/PAL-Host-ABI.md
  2. 9 1
      LibOS/shim/include/shim_internal.h
  3. 2 2
      LibOS/shim/include/shim_thread.h
  4. 1 2
      LibOS/shim/src/bookkeep/shim_signal.c
  5. 2 2
      LibOS/shim/src/elf/shim_rtld.c
  6. 2 2
      LibOS/shim/src/ipc/shim_ipc_child.c
  7. 3 2
      LibOS/shim/src/ipc/shim_ipc_helper.c
  8. 14 7
      LibOS/shim/src/shim_async.c
  9. 7 10
      LibOS/shim/src/shim_init.c
  10. 1 1
      LibOS/shim/src/sys/shim_clone.c
  11. 34 25
      LibOS/shim/src/sys/shim_exit.c
  12. 33 12
      LibOS/shim/src/sys/shim_futex.c
  13. 1 1
      LibOS/shim/test/regression/futex.manifest.template
  14. 2 0
      LibOS/shim/test/regression/manifest.template
  15. 1 1
      Pal/regression/Event.c
  16. 1 1
      Pal/regression/Thread.c
  17. 1 1
      Pal/regression/Thread2.c
  18. 1 1
      Pal/src/db_rtld.c
  19. 2 2
      Pal/src/db_threading.c
  20. 1 1
      Pal/src/host/FreeBSD/db_exception.c
  21. 1 1
      Pal/src/host/FreeBSD/db_exception2.c
  22. 6 2
      Pal/src/host/FreeBSD/db_threading.c
  23. 2 2
      Pal/src/host/Linux-SGX/db_exception.c
  24. 8 3
      Pal/src/host/Linux-SGX/db_threading.c
  25. 4 3
      Pal/src/host/Linux-SGX/enclave_ecalls.c
  26. 9 0
      Pal/src/host/Linux-SGX/enclave_entry.S
  27. 1 0
      Pal/src/host/Linux-SGX/generated-offsets.c
  28. 1 0
      Pal/src/host/Linux-SGX/sgx_tls.h
  29. 1 0
      Pal/src/host/Linux/clone-x86_64.S
  30. 2 2
      Pal/src/host/Linux/db_exception.c
  31. 91 15
      Pal/src/host/Linux/db_threading.c
  32. 1 1
      Pal/src/host/Skeleton/db_threading.c
  33. 1 2
      Pal/src/pal.h
  34. 1 1
      Pal/src/pal_internal.h
  35. 1 1
      Pal/test/Event.c
  36. 1 1
      Pal/test/Fork.c

+ 4 - 2
Documentation/oldwiki/PAL-Host-ABI.md

@@ -432,9 +432,11 @@ This API yields the current thread such that the host scheduler can reschedule i
 
 #### DkThreadExit
 
-    void DkThreadExit(void);
+    void DkThreadExit(PAL_PTR clear_child_tid);
 
-This API terminates the current thread.
+This API terminates the current thread. `clear_child_tid` is the pointer to memory that is erased
+on thread exit to notify LibOS (which in turn notifies the parent thread if any); if
+`clear_child_tid` is NULL, then PAL doesn't do the clearing.
 
 #### DkThreadResume
 

+ 9 - 1
LibOS/shim/include/shim_internal.h

@@ -770,7 +770,7 @@ extern void * __load_address, * __load_address_end;
 extern void * __code_address, * __code_address_end;
 
 /* cleanup and terminate process, preserve exit code if err == 0 */
-int shim_clean (int err);
+noreturn void shim_clean_and_exit(int err);
 
 unsigned long parse_int (const char * str);
 
@@ -797,6 +797,14 @@ void set_rlimit_cur(int resource, uint64_t rlim);
 
 int object_wait_with_retry(PAL_HANDLE handle);
 
+/* this struct is passed as the second argument to release_clear_child_id() */
+struct clear_child_tid_struct {
+    int* clear_child_tid;         /* passed to LibOS's clone() from user app */
+    int  clear_child_tid_val_pal; /* ptr to it is passed to PAL's DkThreadExit() */
+};
+
+void release_clear_child_id(IDTYPE caller, void* clear_child_tids);
+
 #ifdef __x86_64__
 #define __SWITCH_STACK(stack_top, func, arg)                    \
     do {                                                        \

+ 2 - 2
LibOS/shim/include/shim_thread.h

@@ -314,10 +314,10 @@ void set_handle_map (struct shim_thread * thread,
 }
 
 /* shim exit callback */
-int thread_exit (struct shim_thread * self, bool send_ipc);
+int thread_exit(struct shim_thread* self, bool send_ipc, int** clear_child_tid_pal_ptr);
 /* If the process was killed by a signal, pass it in the second
  *  argument, else pass zero */
-int try_process_exit (int error_code, int term_signal);
+noreturn void thread_or_process_exit(int error_code, int term_signal);
 
 /* thread cloning helpers */
 struct shim_clone_args {

+ 1 - 2
LibOS/shim/src/bookkeep/shim_signal.c

@@ -829,8 +829,7 @@ static void sighandler_kill (int sig, siginfo_t * info, void * ucontext)
         }
     }
 
-    try_process_exit(0, sig);
-    DkThreadExit();
+    thread_or_process_exit(0, sig);
 }
 
 static void sighandler_core (int sig, siginfo_t * info, void * ucontext)

+ 2 - 2
LibOS/shim/src/elf/shim_rtld.c

@@ -1562,7 +1562,7 @@ noreturn void execute_elf_object(struct shim_handle* exec, int* argcp, const cha
     int ret = vdso_map_init();
     if (ret < 0) {
         SYS_PRINTF("Could not initialize vDSO (error code = %d)", ret);
-        shim_clean(ret);
+        shim_clean_and_exit(ret);
     }
 
     struct link_map* exec_map = __search_map_by_handle(exec);
@@ -1601,7 +1601,7 @@ noreturn void execute_elf_object(struct shim_handle* exec, int* argcp, const cha
     ret               = DkRandomBitsRead((PAL_PTR)random, 16);
     if (ret < 0) {
         debug("execute_elf_object: DkRandomBitsRead failed.\n");
-        DkThreadExit();
+        DkThreadExit(/*clear_child_tid=*/NULL);
     }
     auxp[5].a_un.a_val = random;
 

+ 2 - 2
LibOS/shim/src/ipc/shim_ipc_child.c

@@ -80,7 +80,7 @@ static int child_thread_exit(struct shim_thread* thread, void* arg, bool* unlock
             /* remote thread is "virtually" exited: SIGCHLD is generated for
              * the parent thread and exit events are arranged for subsequent
              * wait4(). */
-            thread_exit(thread, false);
+            thread_exit(thread, /*send_ipc=*/false, /*clear_child_tid_pal_ptr*/NULL);
             goto out;
         }
     }
@@ -207,7 +207,7 @@ int ipc_cld_exit_callback(struct shim_ipc_msg* msg, struct shim_ipc_port* port)
 
         /* Remote thread is "virtually" exited: SIGCHLD is generated for the
          * parent thread and exit events are arranged for subsequent wait4(). */
-        ret = thread_exit(thread, /*send_ipc=*/false);
+        ret = thread_exit(thread, /*send_ipc=*/false, /*clear_child_tid_pal_ptr*/NULL);
         put_thread(thread);
     } else {
         /* Uncommon case: remote child thread was already exited and deleted

+ 3 - 2
LibOS/shim/src/ipc/shim_ipc_helper.c

@@ -780,10 +780,11 @@ noreturn static void shim_ipc_helper(void* dummy) {
     free(object_list);
     free(palhandle_list);
 
+    __disable_preempt(self->shim_tcb);
     put_thread(self);
     debug("IPC helper thread terminated\n");
 
-    DkThreadExit();
+    DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 static void shim_ipc_helper_prepare(void* arg) {
@@ -803,7 +804,7 @@ static void shim_ipc_helper_prepare(void* arg) {
     if (notme || !stack) {
         free(stack);
         put_thread(self);
-        DkThreadExit();
+        DkThreadExit(/*clear_child_tid=*/NULL);
         return;
     }
 

+ 14 - 7
LibOS/shim/src/shim_async.c

@@ -85,7 +85,7 @@ int64_t install_async_event(PAL_HANDLE object, uint64_t time,
 
     lock(&async_helper_lock);
 
-    if (!object) {
+    if (callback != &release_clear_child_id && !object) {
         /* This is alarm() or setitimer() emulation, treat both according to
          * alarm() syscall semantics: cancel any pending alarm/timer. */
         struct async_event * tmp, * n;
@@ -153,7 +153,7 @@ static void shim_async_helper(void * arg) {
 
     if (notme) {
         put_thread(self);
-        DkThreadExit();
+        DkThreadExit(/*clear_child_tid=*/NULL);
         return;
     }
 
@@ -206,9 +206,16 @@ static void shim_async_helper(void * arg) {
 
         struct async_event * tmp, * n;
         LISTP_FOR_EACH_ENTRY_SAFE(tmp, n, &async_list, list) {
-            /* First check if this event was triggered; note that IO events
-             * stay in the list whereas alarms/timers are fired only once. */
-            if (polled && tmp->object == polled) {
+            /* First check if this event was triggered; there are three types:
+             *   1. Exited child:  trigger callback and remove from the list;
+             *   2. IO events:     trigger callback and keep in the list;
+             *   3. alarms/timers: trigger callback and remove from the list. */
+            if (tmp->callback == &release_clear_child_id) {
+                debug("Child exited, notifying parent if any\n");
+                LISTP_DEL(tmp, &async_list, list);
+                LISTP_ADD_TAIL(tmp, &triggered, list);
+                continue;
+            } else if (polled && tmp->object == polled) {
                 debug("Async IO event triggered at %lu\n", now);
                 unlock(&async_helper_lock);
                 /* FIXME: potential race condition when
@@ -216,7 +223,6 @@ static void shim_async_helper(void * arg) {
                  * correctly implemented. tmp can be freed at the same
                  * time. */
                 tmp->callback(tmp->caller, tmp->arg);
-
                 /* async_list may be changed because async_helper_lock is
                  * released; list traverse cannot be continued. */
                 polled = NULL;
@@ -288,11 +294,12 @@ static void shim_async_helper(void * arg) {
         polled = DkObjectsWaitAny(object_num + 1, object_list, sleep_time);
     }
 
+    __disable_preempt(self->shim_tcb);
     put_thread(self);
     debug("Async helper thread terminated\n");
     free(object_list);
 
-    DkThreadExit();
+    DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 /* this should be called with the async_helper_lock held */

+ 7 - 10
LibOS/shim/src/shim_init.c

@@ -1136,18 +1136,16 @@ noreturn void shim_terminate (int err)
     debug("teminating the whole process (%d)\n", err);
 
     /* do last clean-up of the process */
-    shim_clean(err);
-
-    DkProcessExit(err);
+    shim_clean_and_exit(err);
 }
 
 /* cleanup and terminate process, preserve exit code if err == 0 */
-int shim_clean (int err)
-{
-    /* preventing multiple cleanup, this is mostly caused by
-       assertion in shim_clean */
-    if (atomic_inc_return(&in_terminate) > 1)
-        return 0;
+noreturn void shim_clean_and_exit(int err) {
+    if (atomic_inc_return(&in_terminate) > 1) {
+        while (true) {
+            /* nothing */
+        }
+    }
 
     if (err != 0)
         cur_process.exit_code = err;
@@ -1191,7 +1189,6 @@ int shim_clean (int err)
     debug("process %u exited with status %d\n", cur_process.vmid & 0xFFFF, cur_process.exit_code);
     MASTER_LOCK();
     DkProcessExit(cur_process.exit_code);
-    return 0;
 }
 
 int message_confirm (const char * message, const char * options)

+ 1 - 1
LibOS/shim/src/sys/shim_clone.c

@@ -288,7 +288,7 @@ int shim_do_clone (int flags, void * user_stack_addr, int * parent_tidptr,
 
     if (flags & CLONE_CHILD_CLEARTID)
         /* Implemented in shim_futex.c: release_clear_child_id */
-        thread->clear_child_tid = parent_tidptr;
+        thread->clear_child_tid = child_tidptr;
 
     unsigned long fs_base = 0;
     if (flags & CLONE_SETTLS) {

+ 34 - 25
LibOS/shim/src/sys/shim_exit.c

@@ -40,10 +40,7 @@
 
 void release_robust_list (struct robust_list_head * head);
 
-void release_clear_child_id (int * clear_child_tid);
-
-int thread_exit(struct shim_thread * self, bool send_ipc)
-{
+int thread_exit(struct shim_thread* self, bool send_ipc, int** clear_child_tid_pal_ptr) {
     bool sent_exit_msg = false;
 
     /* Chia-Che: Broadcast exit message as early as possible,
@@ -122,27 +119,43 @@ int thread_exit(struct shim_thread * self, bool send_ipc)
     if (robust_list)
         release_robust_list(robust_list);
 
-    if (self->clear_child_tid)
-        release_clear_child_id (self->clear_child_tid);
+    if (parent && self->in_vm && self->clear_child_tid) {
+        /* ask Async Helper thread to wake up parent when this child thread finally exits;
+         * we must alloc clear_child_tids on heap instead of this thread's stack; it is
+         * freed in release_clear_child_id() */
+        struct clear_child_tid_struct* clear_child_tids = malloc(sizeof(*clear_child_tids));
+        if (clear_child_tids) {
+            clear_child_tids->clear_child_tid         = self->clear_child_tid;
+            clear_child_tids->clear_child_tid_val_pal = 1; /* any non-zero value suffices */
+            install_async_event(NULL, 0, &release_clear_child_id, clear_child_tids);
+
+            if (clear_child_tid_pal_ptr) {
+                /* caller wants to performs DkThreadExit() and needs to know which address
+                 * PAL must set to inform the Async Helper thread */
+                *clear_child_tid_pal_ptr = &clear_child_tids->clear_child_tid_val_pal;
+            }
+        }
+    }
 
     DkEventSet(self->exit_event);
     return 0;
 }
 
 /* note that term_signal argument may contain WCOREDUMP bit (0x80) */
-int try_process_exit (int error_code, int term_signal)
-{
+noreturn void thread_or_process_exit(int error_code, int term_signal) {
     struct shim_thread * cur_thread = get_cur_thread();
 
     cur_thread->exit_code = -error_code;
     cur_process.exit_code = term_signal ? term_signal : error_code;
     cur_thread->term_signal = term_signal;
 
+    int* clear_child_tid_pal = NULL;
     if (cur_thread->in_vm)
-        thread_exit(cur_thread, true);
+        thread_exit(cur_thread, true, &clear_child_tid_pal);
 
-    if (check_last_thread(cur_thread))
-        return 0;
+    if (check_last_thread(cur_thread)) {
+        DkThreadExit(clear_child_tid_pal);
+    }
 
     struct shim_thread * async_thread = terminate_async_helper();
     if (async_thread)
@@ -160,8 +173,7 @@ int try_process_exit (int error_code, int term_signal)
          */
         put_thread(ipc_thread); /* free resources of the thread */
 
-    shim_clean(0);
-    return 0;
+    shim_clean_and_exit(0);
 }
 
 noreturn int shim_do_exit_group (int error_code)
@@ -185,7 +197,7 @@ noreturn int shim_do_exit_group (int error_code)
 #ifndef ALIAS_VFORK_AS_FORK
     if (cur_thread->dummy) {
         cur_thread->term_signal = 0;
-        thread_exit(cur_thread, true);
+        thread_exit(cur_thread, true, NULL);
         switch_dummy_thread(cur_thread);
     }
 #endif
@@ -193,24 +205,23 @@ noreturn int shim_do_exit_group (int error_code)
     debug("now kill other threads in the process\n");
     do_kill_proc(cur_thread->tgid, cur_thread->tgid, SIGKILL, false);
     /* This loop ensures that the current thread, which issues exit_group(), wins in setting the
-     * process's exit code. try_process_exit() first sets the exit_code before updating the thread's
-     * state to "dead". Once check_last_thread() indicates that the current thread is the last
-     * thread, all the children will already have set thread->exit_code. Hence, this thread's
-     * execution of try_process_exit() gets to determine the final exit_code, which is the desired
-     * outcome. */
+     * process's exit code. thread_or_process_exit() first sets the exit_code before updating the
+     * thread's state to "dead". Once check_last_thread() indicates that the current thread is the
+     * last thread, all the children will already have set thread->exit_code. Hence, this thread's
+     * execution of thread_or_process_exit() gets to determine the final exit_code, which is the
+     * desired outcome. */
     while (check_last_thread(cur_thread)) {
         DkThreadYieldExecution();
     }
 
     debug("now exit the process\n");
-    try_process_exit(error_code, 0);
 
 #ifdef PROFILE
     if (ENTER_TIME)
         SAVE_PROFILE_INTERVAL_SINCE(syscall_exit_group, ENTER_TIME);
 #endif
 
-    DkThreadExit();
+    thread_or_process_exit(error_code, 0);
 }
 
 noreturn int shim_do_exit (int error_code)
@@ -226,17 +237,15 @@ noreturn int shim_do_exit (int error_code)
 #ifndef ALIAS_VFORK_AS_FORK
     if (cur_thread->dummy) {
         cur_thread->term_signal = 0;
-        thread_exit(cur_thread, true);
+        thread_exit(cur_thread, true, NULL);
         switch_dummy_thread(cur_thread);
     }
 #endif
 
-    try_process_exit(error_code, 0);
-
 #ifdef PROFILE
     if (ENTER_TIME)
         SAVE_PROFILE_INTERVAL_SINCE(syscall_exit, ENTER_TIME);
 #endif
 
-    DkThreadExit();
+    thread_or_process_exit(error_code, 0);
 }

+ 33 - 12
LibOS/shim/src/sys/shim_futex.c

@@ -427,40 +427,61 @@ void release_robust_list(struct robust_list_head* head) {
     }
 }
 
-void release_clear_child_id(int* clear_child_tid) {
-    debug("clear child tid at %p\n", clear_child_tid);
-    *clear_child_tid = 0;
+/* Function is called by Async Helper thread to wait on clear_child_tid_val_pal to be set to 0
+ * (PAL does it when child thread finally exits). Next, *clear_child_tid is set to 0 and parent
+ * threads are woken up. Since it is a callback to Async Helper thread, it must follow the
+ * `void (*callback) (IDTYPE caller, void * arg)` signature even though we don't use caller. */
+void release_clear_child_id(IDTYPE caller, void* clear_child_tids) {
+    __UNUSED(caller);
+
+    struct clear_child_tid_struct* child = (struct clear_child_tid_struct*)clear_child_tids;
+    if (!child || !child->clear_child_tid)
+        goto out;
+
+    /* wait on clear_child_tid_val_pal; this signals that PAL layer exited child thread */
+    while (__atomic_load_n(&child->clear_child_tid_val_pal, __ATOMIC_RELAXED) != 0) {
+        __asm__ volatile ("pause");
+    }
+
+    /* child thread exited, now parent can wake up; note that PAL layer can't set clear_child_tid
+     * itself, because parent thread could spuriously wake up, notice 0 on clear_child_tid, and
+     * continue its execution without waiting for this function to succeed first */
+    __atomic_store_n(child->clear_child_tid, 0, __ATOMIC_RELAXED);
 
+    /* at this point, child thread finally exited, can wake up parents if any */
     create_lock_runtime(&futex_list_lock);
 
     struct shim_futex_handle* tmp;
     struct shim_futex_handle* futex = NULL;
-    lock(&futex_list_lock);
 
+    lock(&futex_list_lock);
     LISTP_FOR_EACH_ENTRY(tmp, &futex_list, list) {
-        if (tmp->uaddr == (void*)clear_child_tid) {
+        if (tmp->uaddr == (void*)child->clear_child_tid) {
             futex = tmp;
             break;
         }
     }
-
     unlock(&futex_list_lock);
 
-    if (!futex)
-        return;
+    if (!futex) {
+        /* no parent threads waiting on this child to exit */
+        goto out;
+    }
 
+    debug("release futex at %p\n", child->clear_child_tid);
     struct futex_waiter* waiter;
     struct futex_waiter* wtmp;
     struct shim_handle* hdl = container_of(futex, struct shim_handle, info.futex);
+
     get_handle(hdl);
     lock(&hdl->lock);
-
-    debug("release futex at %p\n", clear_child_tid);
-    *clear_child_tid = 0;
     LISTP_FOR_EACH_ENTRY_SAFE(waiter, wtmp, &futex->waiters, list) {
+        /* wake up every parent waiting on this child */
         del_futex_waiter_wakeup(waiter, futex);
     }
-
     unlock(&hdl->lock);
     put_handle(hdl);
+
+out:
+    free(child);
 }

+ 1 - 1
LibOS/shim/test/regression/futex.manifest.template

@@ -19,4 +19,4 @@ net.rules.2 = 0.0.0.0:0-65535:127.0.0.1:8000
 sgx.trusted_files.ld = file:../../../../Runtime/ld-linux-x86-64.so.2
 sgx.trusted_files.libc = file:../../../../Runtime/libc.so.6
 sgx.trusted_files.libpthread = file:../../../../Runtime/libpthread.so.0
-sgx.thread_num = 4
+sgx.thread_num = 6

+ 2 - 0
LibOS/shim/test/regression/manifest.template

@@ -38,3 +38,5 @@ sgx.trusted_children.victim = file:exec_victim.sig
 sgx.allow_file_creation = 1
 
 sgx.allowed_files.tmp_dir = file:tmp/
+
+sgx.thread_num = 6

+ 1 - 1
Pal/regression/Event.c

@@ -14,7 +14,7 @@ int thread2_run(void* args) {
     pal_printf("Sending event...\n");
     DkEventSet(event1);
     pal_printf("End of second thread.\n");
-    DkThreadExit();
+    DkThreadExit(/*clear_child_tid=*/NULL);
 
     return 0;
 }

+ 1 - 1
Pal/regression/Thread.c

@@ -27,7 +27,7 @@ int callback1(void* args) {
 
     count1 = 100;
     __asm__ volatile("nop" ::: "memory");
-    DkThreadExit();
+    DkThreadExit(/*clear_child_tid=*/NULL);
     count1 = 101;
     __asm__ volatile("nop" ::: "memory");
 

+ 1 - 1
Pal/regression/Thread2.c

@@ -28,7 +28,7 @@ int thread3_run(void* args) {
 
     // Ensure that the compiler can't know that this should never return.
     if (dummy_true) {
-        DkThreadExit();
+        DkThreadExit(/*clear_child_tid=*/NULL);
     }
 
     thread3_exit_ok = false;

+ 1 - 1
Pal/src/db_rtld.c

@@ -1355,5 +1355,5 @@ noreturn void start_execution(const char** arguments, const char** environs) {
     if (exec_map)
         CALL_ENTRY(exec_map, cookies);
 
-    _DkThreadExit();
+    _DkThreadExit(/*clear_child_tid=*/NULL);
 }

+ 2 - 2
Pal/src/db_threading.c

@@ -71,9 +71,9 @@ void DkThreadYieldExecution(void) {
 
 /* PAL call DkThreadExit: simply exit the current thread
    no matter what */
-noreturn void DkThreadExit(void) {
+noreturn void DkThreadExit(PAL_PTR clear_child_tid) {
     ENTER_PAL_CALL(DkThreadExit);
-    _DkThreadExit();
+    _DkThreadExit((int*)clear_child_tid);
     _DkRaiseFailure(PAL_ERROR_NOTKILLABLE);
     while (true)
         /* nothing */;

+ 1 - 1
Pal/src/host/FreeBSD/db_exception.c

@@ -391,7 +391,7 @@ static void _DkTerminateSighandler (int signum, siginfo_t * info,
         return;
 
     if (!_DkGenericSignalHandle(event_num, NULL, frame, uc, eframe))
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 static void _DkPipeSighandler (int signum, siginfo_t * info,

+ 1 - 1
Pal/src/host/FreeBSD/db_exception2.c

@@ -314,7 +314,7 @@ static void _DkTerminateSighandler (int signum, siginfo_t * info,
         return;
 
     if (!_DkGenericSignalHandle(event_num, NULL, frame, uc, eframe))
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 static void _DkPipeSighandler (int signum, siginfo_t * info,

+ 6 - 2
Pal/src/host/FreeBSD/db_threading.c

@@ -113,8 +113,12 @@ void _DkThreadYieldExecution (void)
 }
 
 /* _DkThreadExit for internal use: Thread exiting */
-noreturn void _DkThreadExit (void)
-{
+noreturn void _DkThreadExit(int* clear_child_tid) {
+    if (clear_child_tid) {
+        /* thread is ready to exit, must inform LibOS by setting *clear_child_tid to 0;
+         * async helper thread in LibOS is waiting on this to wake up parent */
+        __atomic_store_n(clear_child_tid, 0, __ATOMIC_RELAXED);
+    }
     INLINE_SYSCALL(exit, 1, 0);
 }
 

+ 2 - 2
Pal/src/host/Linux-SGX/db_exception.c

@@ -224,7 +224,7 @@ void _DkExceptionHandler (unsigned int exit_info, sgx_cpu_context_t * uc)
         while (true)
             __asm__ volatile("pause");
 #endif
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
     }
 
     PAL_CONTEXT ctx;
@@ -350,5 +350,5 @@ void _DkHandleExternalEvent (PAL_NUM event, sgx_cpu_context_t * uc)
 
     if (!_DkGenericSignalHandle(event, 0, frame, NULL)
         && event != PAL_EVENT_RESUME)
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
 }

+ 8 - 3
Pal/src/host/Linux-SGX/db_threading.c

@@ -91,7 +91,7 @@ void pal_start_thread (void)
     PAL_TCB* pal_tcb = pal_get_tcb();
     memset(&pal_tcb->libos_tcb, 0, sizeof(pal_tcb->libos_tcb));
     callback((void *) param);
-    _DkThreadExit();
+    _DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 /* _DkThreadCreate for internal use. Create an internal thread
@@ -140,10 +140,15 @@ void _DkThreadYieldExecution (void)
 }
 
 /* _DkThreadExit for internal use: Thread exiting */
-noreturn void _DkThreadExit (void)
-{
+noreturn void _DkThreadExit(int* clear_child_tid) {
     struct pal_handle_thread* exiting_thread = GET_ENCLAVE_TLS(thread);
 
+    /* thread is ready to exit, must inform LibOS by erasing clear_child_tid;
+     * note that we don't do it now (because this thread still occupies SGX
+     * TCS slot) but during handle_thread_reset in assembly code */
+    SET_ENCLAVE_TLS(clear_child_tid, clear_child_tid);
+    static_assert(sizeof(*clear_child_tid) == 4,  "unexpected clear_child_tid size");
+
     /* main thread is not part of the thread_list */
     if(exiting_thread != &pal_control.first_thread->thread) {
         _DkInternalLock(&thread_list_lock);

+ 4 - 3
Pal/src/host/Linux-SGX/enclave_ecalls.c

@@ -50,9 +50,10 @@ void handle_ecall (long ecall_index, void * ecall_args, void * exit_target,
         enclave_top = enclave_base_addr + GET_ENCLAVE_TLS(enclave_size);
     }
 
-    SET_ENCLAVE_TLS(exit_target, exit_target);
-    SET_ENCLAVE_TLS(ustack_top,  untrusted_stack);
-    SET_ENCLAVE_TLS(ustack,      untrusted_stack);
+    SET_ENCLAVE_TLS(exit_target,     exit_target);
+    SET_ENCLAVE_TLS(ustack_top,      untrusted_stack);
+    SET_ENCLAVE_TLS(ustack,          untrusted_stack);
+    SET_ENCLAVE_TLS(clear_child_tid, NULL);
 
     if (atomic_cmpxchg(&enclave_start_called, 0, 1) == 0) {
         // ENCLAVE_START not yet called, so only valid ecall is ENCLAVE_START.

+ 9 - 0
Pal/src/host/Linux-SGX/enclave_entry.S

@@ -117,6 +117,15 @@ enclave_entry:
 .Lhandle_thread_reset:
 	movq $0, %gs:SGX_READY_FOR_EXCEPTIONS
 
+	# At this point, the thread has completely exited from the point of view
+	# of LibOS. We can now set *clear_child_tid to 0, which will trigger
+	# async helper thread in LibOS, who will wake up parent thread if any.
+	cmpq $0, %gs:SGX_CLEAR_CHILD_TID
+	je 1f
+	movq %gs:SGX_CLEAR_CHILD_TID, %rbx
+	movl $0, (%rbx)
+
+1:
 	# Signals are impossible at this point: benign untrusted runtime blocks
 	# all signals (see sgx_ocall_exit()), and even if malicious one doesn't
 	# block them, signals are ignored due to SGX_READY_FOR_EXCEPTIONS = 0.

+ 1 - 0
Pal/src/host/Linux-SGX/generated-offsets.c

@@ -91,6 +91,7 @@ void dummy(void)
     OFFSET(SGX_HEAP_MAX, enclave_tls, heap_max);
     OFFSET(SGX_EXEC_ADDR, enclave_tls, exec_addr);
     OFFSET(SGX_EXEC_SIZE, enclave_tls, exec_size);
+    OFFSET(SGX_CLEAR_CHILD_TID, enclave_tls, clear_child_tid);
 
     /* struct pal_tcb_linux aka PAL_TCB_LINUX */
     OFFSET(PAL_TCB_LINUX_TCS, pal_tcb_linux, tcs);

+ 1 - 0
Pal/src/host/Linux-SGX/sgx_tls.h

@@ -33,6 +33,7 @@ struct enclave_tls {
         void*    heap_max;
         void*    exec_addr;
         uint64_t exec_size;
+        int*     clear_child_tid;
     };
 };
 

+ 1 - 0
Pal/src/host/Linux/clone-x86_64.S

@@ -108,6 +108,7 @@ thread_start:
 	/* movl $SYS_ify(exit), %eax */
 	/* syscall */
 	/* Instead of syscall exit, let's call _DkThreadExit */
+	movq	$0, %rdi
 	call _DkThreadExit
 	cfi_endproc
 

+ 2 - 2
Pal/src/host/Linux/db_exception.c

@@ -250,7 +250,7 @@ static void _DkGenericSighandler (int signum, siginfo_t * info,
             INLINE_SYSCALL(nanosleep, 2, &sleeptime, NULL);
         }
 #endif
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
         return;
     }
 
@@ -293,7 +293,7 @@ static void _DkTerminateSighandler (int signum, siginfo_t * info,
     // Call the event handler. If there is no handler, terminate the thread
     // unless it is a resuming event (then ignore the event).
     if (!_DkGenericSignalHandle(event_num, NULL, uc) && event_num != PAL_EVENT_RESUME)
-        _DkThreadExit();
+        _DkThreadExit(/*clear_child_tid=*/NULL);
 }
 
 static void _DkPipeSighandler (int signum, siginfo_t * info,

+ 91 - 15
Pal/src/host/Linux/db_threading.c

@@ -20,19 +20,19 @@
  * This file contain APIs to create, exit and yield a thread.
  */
 
-#include "pal_defs.h"
-#include "pal_linux_defs.h"
+#include "api.h"
 #include "pal.h"
+#include "pal_debug.h"
+#include "pal_defs.h"
+#include "pal_error.h"
 #include "pal_internal.h"
 #include "pal_linux.h"
-#include "pal_error.h"
-#include "pal_debug.h"
-#include "api.h"
-
+#include "pal_linux_defs.h"
+#include "spinlock.h"
 #include <errno.h>
-#include <linux/signal.h>
 #include <linux/mman.h>
 #include <linux/sched.h>
+#include <linux/signal.h>
 #include <linux/types.h>
 #include <linux/wait.h>
 
@@ -42,6 +42,58 @@
 #include <asm/prctl.h>
 #endif
 
+/* Linux PAL cannot use mmap/unmap to manage thread stacks because this may overlap with
+ * pal_control.user_address. Linux PAL also cannot just use malloc/free because DkThreadExit
+ * needs to use raw system calls and inline asm. Thus, we resort to recycling thread stacks
+ * allocated by previous threads and not used anymore. This still leaks memory but at least
+ * it is bounded by the maximum number of simultaneously executing threads. Note that main
+ * thread is not a part of this mechanism (it only allocates a tiny altstack). */
+struct thread_stack_map_t {
+    void* stack;
+    bool  used;
+};
+
+static struct thread_stack_map_t* g_thread_stack_map = NULL;
+static size_t g_thread_stack_num  = 0;
+static size_t g_thread_stack_size = 0;
+static spinlock_t g_thread_stack_lock = INIT_SPINLOCK_UNLOCKED;
+
+static void* get_thread_stack(void) {
+    void* ret = NULL;
+    spinlock_lock(&g_thread_stack_lock);
+    for (size_t i = 0; i < g_thread_stack_num; i++) {
+        if (!g_thread_stack_map[i].used) {
+            /* found allocated and unused stack -- use it */
+            g_thread_stack_map[i].used = true;
+            ret = g_thread_stack_map[i].stack;
+            goto out;
+        }
+    }
+
+    if (g_thread_stack_num == g_thread_stack_size) {
+        /* realloc g_thread_stack_map to accommodate more objects (includes the very first time) */
+        g_thread_stack_size += 8;
+        struct thread_stack_map_t* tmp = malloc(g_thread_stack_size * sizeof(*tmp));
+        if (!tmp)
+            goto out;
+
+        memcpy(tmp, g_thread_stack_map, g_thread_stack_num * sizeof(*tmp));
+        free(g_thread_stack_map);
+        g_thread_stack_map = tmp;
+    }
+
+    ret = malloc(THREAD_STACK_SIZE + ALT_STACK_SIZE);
+    if (!ret)
+        goto out;
+
+    g_thread_stack_map[g_thread_stack_num].stack = ret;
+    g_thread_stack_map[g_thread_stack_num].used  = true;
+    g_thread_stack_num++;
+out:
+    spinlock_unlock(&g_thread_stack_lock);
+    return ret;
+}
+
 /*
  * pal_thread_init(): An initialization wrapper of a newly-created thread (including
  * the first thread). This function accepts a TCB pointer to be set to the GS register
@@ -85,7 +137,7 @@ int _DkThreadCreate (PAL_HANDLE * handle, int (*callback) (void *),
 {
     int ret = 0;
     PAL_HANDLE hdl = NULL;
-    void * stack = malloc(THREAD_STACK_SIZE + ALT_STACK_SIZE);
+    void* stack = get_thread_stack();
     if (!stack) {
         ret = -ENOMEM;
         goto err;
@@ -184,10 +236,10 @@ void _DkThreadYieldExecution (void)
 }
 
 /* _DkThreadExit for internal use: Thread exiting */
-noreturn void _DkThreadExit (void)
-{
+noreturn void _DkThreadExit(int* clear_child_tid) {
     PAL_TCB_LINUX* tcb = get_tcb_linux();
     PAL_HANDLE handle = tcb->handle;
+    assert(handle);
 
     block_async_signals(true);
     if (tcb->alt_stack) {
@@ -201,13 +253,37 @@ noreturn void _DkThreadExit (void)
         INLINE_SYSCALL(sigaltstack, 2, &ss, NULL);
     }
 
-    if (handle) {
-        // Free the thread stack
-        free(handle->thread.stack);
-        // After this line, needs to exit the thread immediately
+    /* we do not free thread stack but instead mark it as recycled, see get_thread_stack() */
+    spinlock_lock(&g_thread_stack_lock);
+    for (size_t i = 0; i < g_thread_stack_num; i++) {
+        if (g_thread_stack_map[i].stack == handle->thread.stack) {
+            g_thread_stack_map[i].used = false;
+            break;
+        }
     }
+    /* we might still be using the stack we just marked as unused until we enter the asm mode,
+     * so we do not unlock now but rather in asm below */
+
+    /* To make sure the compiler doesn't touch the stack after it was freed, need inline asm:
+     *   1. Unlock g_thread_stack_lock (so that other threads can start re-using this stack)
+     *   2. Set *clear_child_tid = 0 if clear_child_tid != NULL
+     *      (we thus inform LibOS, where async helper thread is waiting on this to wake up parent)
+     *   3. Exit thread */
+    static_assert(sizeof(g_thread_stack_lock) == 4, "unexpected g_thread_stack_lock size");
+    static_assert(sizeof(*clear_child_tid) == 4,  "unexpected clear_child_tid size");
+
+    __asm__ volatile("movl $0, (%%rdx) \n\t"   /* spinlock_unlock(&g_thread_stack_lock) */
+                     "cmpq $0, %%rbx \n\t"     /* check if clear_child_tid != NULL */
+                     "je 1f \n\t"
+                     "movl $0, (%%rbx) \n\t"   /* set *clear_child_tid = 0 */
+                     "1: \n\t"
+                     "syscall \n\t"            /* rdi arg is already prepared, call exit */
+                     : /* no output regs since we don't return from exit */
+                     : "a"(__NR_exit), "D"(0), /* rdi = exit status == 0 */
+                       "d"(&g_thread_stack_lock), "b"(clear_child_tid)
+                     : "cc", "rcx", "r11", "memory"  /* syscall instr clobbers cc, rcx, and r11 */
+    );
 
-    INLINE_SYSCALL(exit, 1, 0);
     while (true) {
         /* nothing */
     }

+ 1 - 1
Pal/src/host/Skeleton/db_threading.c

@@ -45,7 +45,7 @@ void _DkThreadYieldExecution(void) {
 }
 
 /* _DkThreadExit for internal use: Thread exiting */
-noreturn void _DkThreadExit(void) {
+noreturn void _DkThreadExit(int* clear_child_tid) {
     /* needs to be implemented */
     while (true) {
         /* nothing */

+ 1 - 2
Pal/src/pal.h

@@ -423,8 +423,7 @@ DkThreadDelayExecution (PAL_NUM duration);
 void
 DkThreadYieldExecution (void);
 
-noreturn void
-DkThreadExit (void);
+noreturn void DkThreadExit(PAL_PTR clear_child_tid);
 
 PAL_BOL
 DkThreadResume (PAL_HANDLE thread);

+ 1 - 1
Pal/src/pal_internal.h

@@ -294,7 +294,7 @@ PAL_HANDLE _DkBroadcastStreamOpen (void);
 /* DkProcess and DkThread calls */
 int _DkThreadCreate (PAL_HANDLE * handle, int (*callback) (void *),
                      const void * param);
-noreturn void _DkThreadExit (void);
+noreturn void _DkThreadExit(int* clear_child_tid);
 int _DkThreadDelayExecution (unsigned long * duration);
 void _DkThreadYieldExecution (void);
 int _DkThreadResume (PAL_HANDLE threadHandle);

+ 1 - 1
Pal/test/Event.c

@@ -17,7 +17,7 @@ int thread_1(void* args) {
     }
 
     DkEventSet(event1);
-    DkThreadExit();
+    DkThreadExit(/*clear_child_tid=*/NULL);
 
     return 0;
 }

+ 1 - 1
Pal/test/Fork.c

@@ -67,7 +67,7 @@ int main(int argc, char** argv) {
         }
 
         DkVirtualMemoryFree(str, 20);
-        DkThreadExit();
+        DkThreadExit(/*clear_child_tid=*/NULL);
     } else {
         pal_printf("in the parent\n");
         DkThreadDelayExecution(3000);