Browse Source

[Pal/Linux-SGX] Move from using pthread functions to raw clone() syscall

Previously, the Linux-SGX PAL (ab)used pthread functions to create new threads
as part of the clone(CLONE_THREAD) emulation. This is not necessary and in fact
contradicts the idea that PAL should be Glibc/pthread-agnostic. This commit
re-writes all pthread functionality to raw clone() syscall and additional
plumbing of alt stack and GS-based untrusted-PAL TCB. Now the threading
functionality of Linux-SGX PAL is very similar to that of Linux PAL.
Dmitrii Kuvaiskii 4 years ago
parent
commit
74308b3990

+ 2 - 2
Pal/src/host/Linux-SGX/Makefile

@@ -18,7 +18,7 @@ enclave-objs = $(addprefix db_,files devices pipes eventfd sockets streams memor
 	       $(addprefix enclave_,ocalls ecalls framework platform pages untrusted)
 enclave-asm-objs = enclave_entry
 urts-objs = $(addprefix sgx_,enclave framework platform main rtld thread process exception graphene) \
-	    quote/aesm.pb-c
+	    quote/aesm.pb-c clone-x86_64
 urts-asm-objs = sgx_entry
 graphene_lib = .lib/graphene-lib.a
 
@@ -64,7 +64,7 @@ $(addsuffix .o,$(enclave-objs)): quote/generated-cacert.h
 
 pal-sgx: $(addsuffix .o,$(urts-objs) $(urts-asm-objs)) $(graphene_lib)
 	@echo [ host/Linux-SGX/$@ ]
-	@$(CC) $(CFLAGS) -Wl,-z,relro,-z,now -pie $^ -lc -pthread -lprotobuf-c -o $@
+	@$(CC) $(CFLAGS) -Wl,-z,relro,-z,now -pie $^ -lc -lprotobuf-c -o $@
 
 quote/aesm.pb-c.o: quote/aesm.pb-c.c quote/aesm.pb-c.h
 quote/aesm.pb-c.c quote/aesm.pb-c.h: quote/aesm.proto

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

@@ -0,0 +1,115 @@
+/* Copyright (C) 2014 Stony Brook University
+   This file is part of Graphene Library OS.
+
+   Graphene Library OS is free software: you can redistribute it and/or
+   modify it under the terms of the GNU Lesser General Public License
+   as published by the Free Software Foundation, either version 3 of the
+   License, or (at your option) any later version.
+
+   Graphene Library OS is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU Lesser General Public License for more details.
+
+   You should have received a copy of the GNU Lesser General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+/*
+ * clone-x86_64.S
+ *
+ * This file contains architecture-specific implementation of clone
+ * method.
+ * The source code is imported and modified from the GNU C Library.
+ */
+
+/* clone() is even more special than fork() as it mucks with stacks
+   and invokes a function in the right context after its all over.  */
+
+#include <asm/unistd.h>
+#include <asm/errno.h>
+
+#include "sysdep-x86_64.h"
+
+#define CLONE_VM	0x00000100
+#define CLONE_THREAD	0x00010000
+
+/* The userland implementation is:
+   int clone (int (*fn)(void *arg), void *child_stack, int flags, void *arg),
+   the kernel entry is:
+   int clone (long flags, void *child_stack).
+
+   The parameters are passed in register and on the stack from userland:
+   rdi:	fn
+   rsi:	child_stack
+   rdx:	flags
+   rcx:	arg
+   r8:	TID field in parent
+   r9:	thread pointer
+   esp+8: TID field in child
+
+   The kernel expects:
+   rax: system call number
+   rdi: flags
+   rsi: child_stack
+   rdx: TID field in parent
+   r10: TID field in child
+   r8:	thread pointer  */
+
+
+        .text
+ENTRY (clone)
+	/* Sanity check arguments.  */
+	movq	$-EINVAL, %rax
+	testq	%rdi, %rdi		/* no NULL function pointers */
+	/* jz	SYSCALL_ERROR_LABEL */
+	testq	%rsi, %rsi		/* no NULL stack pointers */
+	/* jz	SYSCALL_ERROR_LABEL */
+
+	/* Insert the argument onto the new stack.  */
+	subq	$16, %rsi
+	movq	%rcx, 8(%rsi)
+
+	/* Save the function pointer.  It will be popped off in the
+	   child in the ebx frobbing below.  */
+	movq	%rdi, 0(%rsi)
+
+	/* Do the system call.  */
+	movq	%rdx, %rdi
+	movq	%r8, %rdx
+	movq	%r9, %r8
+	movq	8(%rsp), %r10
+	movl	$SYS_ify(clone), %eax
+
+	/* End FDE now, because in the child the unwind info will be
+	   wrong.  */
+	cfi_endproc
+	syscall
+
+	testq	%rax, %rax
+	/* jl	SYSCALL_ERROR_LABEL */
+	jz	thread_start
+	ret
+
+ENTRY (thread_start)
+thread_start:
+	cfi_startproc
+	/* Clearing frame pointer is insufficient, use CFI.  */
+	cfi_undefined (rip)
+	/* Clear the frame pointer.  The ABI suggests this be done, to mark
+	   the outermost frame obviously.  */
+	xorl	%ebp, %ebp
+
+	/* Set up arguments for the function call.  */
+	popq	%rax		/* Function to call.  */
+	popq	%rdi		/* Argument.  */
+	call	*%rax
+	/* Call exit with return value from function call. */
+	/* movq %rax, %rdi */
+	/* movl $SYS_ify(exit), %eax */
+	/* syscall */
+	/* Instead of syscall exit, let's call thread_exit */
+	call thread_exit
+	cfi_endproc
+
+	cfi_startproc
+END (clone)

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

@@ -92,6 +92,9 @@ void dummy(void)
     OFFSET(SGX_EXEC_ADDR, enclave_tls, exec_addr);
     OFFSET(SGX_EXEC_SIZE, enclave_tls, exec_size);
 
+    /* struct pal_tcb_linux aka PAL_TCB_LINUX */
+    OFFSET(PAL_TCB_LINUX_TCS, pal_tcb_linux, tcs);
+
     /* sgx_arch_tcs_t */
     OFFSET_T(TCS_FLAGS, sgx_arch_tcs_t, flags);
     OFFSET_T(TCS_OSSA, sgx_arch_tcs_t, ossa);

+ 6 - 1
Pal/src/host/Linux-SGX/pal_linux.h

@@ -200,7 +200,7 @@ int sgx_create_process(const char* uri, int nargs, const char** args, int* retfd
 # endif
 
 # define ARCH_VFORK()                                                       \
-    (current_enclave->pal_sec.in_gdb ?                                      \
+    (pal_enclave.pal_sec.in_gdb ?                                           \
      INLINE_SYSCALL(clone, 4, CLONE_VM|CLONE_VFORK|SIGCHLD, 0, NULL, NULL) :\
      INLINE_SYSCALL(clone, 4, CLONE_VM|CLONE_VFORK, 0, NULL, NULL))
 #else
@@ -233,4 +233,9 @@ int sgx_create_process(const char* uri, int nargs, const char** args, int* retfd
     do { if ((class) & DBG_LEVEL) pal_printf(fmt); } while (0)
 #endif
 
+#ifndef IN_ENCLAVE
+int clone(int (*__fn) (void* __arg), void* __child_stack,
+          int __flags, const void* __arg, ...);
+#endif
+
 #endif /* PAL_LINUX_H */

+ 3 - 0
Pal/src/host/Linux-SGX/pal_linux_defs.h

@@ -1,6 +1,9 @@
 #ifndef PAL_LINUX_DEFS_H
 #define PAL_LINUX_DEFS_H
 
+#define THREAD_STACK_SIZE (PRESET_PAGESIZE * 2)
+#define ALT_STACK_SIZE    PRESET_PAGESIZE
+
 #define ENCLAVE_HIGH_ADDRESS    0x800000000
 #define SSAFRAMENUM         2
 #define MEMORY_GAP          PRESET_PAGESIZE

+ 6 - 10
Pal/src/host/Linux-SGX/sgx_enclave.c

@@ -39,12 +39,8 @@ static int sgx_ocall_exit(void* pms)
     block_async_signals(true);
     ecall_thread_reset();
 
-    /* threads created with pthread_create() must exit with pthread_exit() */
-    if (!unmap_tcs())
-        INLINE_SYSCALL(exit, 1, (int)ms->ms_exitcode);
-    else
-        thread_exit(&ms->ms_exitcode);
-
+    unmap_tcs();
+    thread_exit((int)ms->ms_exitcode);
     return 0;
 }
 
@@ -442,7 +438,7 @@ static int sgx_ocall_sock_recv(void * pms)
     struct sockaddr * addr = ms->ms_addr;
     socklen_t addrlen = ms->ms_addr ? ms->ms_addrlen : 0;
 
-    if (ms->ms_sockfd == PAL_SEC()->mcast_srv)
+    if (ms->ms_sockfd == pal_enclave.pal_sec.mcast_srv)
         addr = NULL;
 
     ret = INLINE_SYSCALL(recvfrom, 6,
@@ -464,10 +460,10 @@ static int sgx_ocall_sock_send(void * pms)
     socklen_t addrlen = ms->ms_addr ? ms->ms_addrlen : 0;
     struct sockaddr_in mcast_addr;
 
-    if (ms->ms_sockfd == PAL_SEC()->mcast_srv) {
+    if (ms->ms_sockfd == pal_enclave.pal_sec.mcast_srv) {
         mcast_addr.sin_family = AF_INET;
         inet_pton4(MCAST_GROUP, sizeof(MCAST_GROUP),  &mcast_addr.sin_addr.s_addr);
-        mcast_addr.sin_port = htons(PAL_SEC()->mcast_port);
+        mcast_addr.sin_port = htons(pal_enclave.pal_sec.mcast_port);
         addr = (struct sockaddr *) &mcast_addr;
         addrlen = sizeof(struct sockaddr_in);
     }
@@ -750,7 +746,7 @@ int ecall_enclave_start (char * args, size_t args_size, char * env, size_t env_s
     ms.ms_args_size = args_size;
     ms.ms_env = env;
     ms.ms_env_size = env_size;
-    ms.ms_sec_info = PAL_SEC();
+    ms.ms_sec_info = &pal_enclave.pal_sec;
     EDEBUG(ECALL_ENCLAVE_START, &ms);
     return sgx_ecall(ECALL_ENCLAVE_START, &ms);
 }

+ 1 - 2
Pal/src/host/Linux-SGX/sgx_entry.S

@@ -18,8 +18,7 @@ sgx_ecall:
 
 .Ldo_ecall:
 	# RBX has to be the TCS of the thread
-	movq current_tcs@gottpoff(%rip), %rbx
-	movq %fs:(%rbx), %rbx
+	movq %gs:PAL_TCB_LINUX_TCS, %rbx
 
 	# RCX has to be the AEP (Asynchronous Exit Pointer)
 	leaq async_exit_pointer(%rip), %rcx

+ 6 - 15
Pal/src/host/Linux-SGX/sgx_internal.h

@@ -59,7 +59,7 @@ uint16_t htons (uint16_t shortval);
 uint32_t ntohl (uint32_t longval);
 uint16_t ntohs (uint16_t shortval);
 
-struct pal_enclave {
+extern struct pal_enclave {
     /* attributes */
     unsigned long baseaddr;
     unsigned long size;
@@ -77,7 +77,7 @@ struct pal_enclave {
 
     /* security information */
     struct pal_sec pal_sec;
-};
+} pal_enclave;
 
 int open_gsgx (void);
 bool is_wrfsbase_supported (void);
@@ -120,19 +120,10 @@ int interrupt_thread (void * tcs);
 int clone_thread (void);
 
 void create_tcs_mapper (void * tcs_base, unsigned int thread_num);
-void map_tcs(unsigned int tid, bool created_by_pthread);
-bool unmap_tcs(void);
-void thread_exit(void* rv);
-
-extern __thread struct pal_enclave * current_enclave;
-
-#define PAL_SEC() (&current_enclave->pal_sec)
-
-extern __thread sgx_arch_tcs_t * current_tcs
-            __attribute__((tls_model ("initial-exec")));
-
-extern __thread unsigned long debug_register
-            __attribute__((tls_model ("initial-exec")));
+int pal_thread_init(void* tcbptr);
+void map_tcs(unsigned int tid);
+void unmap_tcs(void);
+void thread_exit(int status);
 
 uint64_t sgx_edbgrd (void * addr);
 void sgx_edbgwr (void * addr, uint64_t data);

+ 25 - 18
Pal/src/host/Linux-SGX/sgx_main.c

@@ -23,6 +23,8 @@ unsigned long pagesize  = PRESET_PAGESIZE;
 unsigned long pagemask  = ~(PRESET_PAGESIZE - 1);
 unsigned long pageshift = PRESET_PAGESIZE - 1;
 
+struct pal_enclave pal_enclave;
+
 static inline
 char * alloc_concat(const char * p, size_t plen,
                     const char * s, size_t slen)
@@ -934,8 +936,19 @@ static int load_enclave (struct pal_enclave * enclave,
     if (ret < 0)
         return ret;
 
-    current_enclave = enclave;
-    map_tcs(INLINE_SYSCALL(gettid, 0), /* created_by_pthread=*/false);
+    void* alt_stack = (void*)INLINE_SYSCALL(mmap, 6, NULL, ALT_STACK_SIZE,
+                                            PROT_READ | PROT_WRITE,
+                                            MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
+    if (IS_ERR_P(alt_stack))
+        return -ENOMEM;
+
+    /* initialize TCB at the top of the alternative stack */
+    PAL_TCB_LINUX* tcb = alt_stack + ALT_STACK_SIZE - sizeof(PAL_TCB_LINUX);
+    tcb->common.self   = &tcb->common;
+    tcb->alt_stack     = alt_stack;
+    tcb->stack         = NULL;  /* main thread uses the stack provided by Linux */
+    tcb->tcs           = NULL;  /* initialized by child thread */
+    pal_thread_init(tcb);
 
     /* start running trusted PAL */
     ecall_enclave_start(args, args_size, env, env_size);
@@ -947,6 +960,7 @@ static int load_enclave (struct pal_enclave * enclave,
 #endif
 
     unmap_tcs();
+    INLINE_SYSCALL(munmap, 2, alt_stack, ALT_STACK_SIZE);
     INLINE_SYSCALL(exit, 0);
     return 0;
 }
@@ -964,13 +978,7 @@ int main (int argc, char ** argv, char ** envp)
     argc--;
     argv++;
 
-    struct pal_enclave * enclave = malloc(sizeof(struct pal_enclave));
-    if (!enclave)
-        return -ENOMEM;
-
-    memset(enclave, 0, sizeof(struct pal_enclave));
-
-    int is_child = sgx_init_child_process(&enclave->pal_sec);
+    int is_child = sgx_init_child_process(&pal_enclave.pal_sec);
     if (is_child < 0) {
         ret = is_child;
         goto out;
@@ -989,7 +997,7 @@ int main (int argc, char ** argv, char ** envp)
             exec_uri = alloc_concat("file:", -1, argv[0], -1);
         }
     } else {
-        exec_uri = alloc_concat(enclave->pal_sec.exec_name, -1, NULL, -1);
+        exec_uri = alloc_concat(pal_enclave.pal_sec.exec_name, -1, NULL, -1);
     }
 
     if (!exec_uri) {
@@ -1093,17 +1101,16 @@ int main (int argc, char ** argv, char ** envp)
     char * env = envp[0];
     size_t env_size = envc > 0 ? (envp[envc - 1] - envp[0]) + strlen(envp[envc - 1]) + 1: 0;
 
-    ret = load_enclave(enclave, manifest_fd, manifest_uri, exec_uri, args, args_size, env, env_size,
+    ret = load_enclave(&pal_enclave, manifest_fd, manifest_uri, exec_uri, args, args_size, env, env_size,
                        exec_uri_inferred);
 
 out:
-    if (enclave->exec >= 0)
-        INLINE_SYSCALL(close, 1, enclave->exec);
-    if (enclave->sigfile >= 0)
-        INLINE_SYSCALL(close, 1, enclave->sigfile);
-    if (enclave->token >= 0)
-        INLINE_SYSCALL(close, 1, enclave->token);
-    free(enclave);
+    if (pal_enclave.exec >= 0)
+        INLINE_SYSCALL(close, 1, pal_enclave.exec);
+    if (pal_enclave.sigfile >= 0)
+        INLINE_SYSCALL(close, 1, pal_enclave.sigfile);
+    if (pal_enclave.token >= 0)
+        INLINE_SYSCALL(close, 1, pal_enclave.token);
     if (!IS_ERR(fd))
         INLINE_SYSCALL(close, 1, fd);
     free(exec_uri);

+ 1 - 1
Pal/src/host/Linux-SGX/sgx_process.c

@@ -130,7 +130,7 @@ int sgx_create_process(const char* uri, int nargs, const char** args, int * retf
 
     int pipe_in = proc_fds[1][0], pipe_out = proc_fds[1][1];
 
-    struct pal_sec * pal_sec = &current_enclave->pal_sec;
+    struct pal_sec * pal_sec = &pal_enclave.pal_sec;
     struct proc_args proc_args;
     memcpy(proc_args.exec_name, uri, sizeof(PAL_SEC_STR));
     proc_args.instance_id   = pal_sec->instance_id;

+ 144 - 39
Pal/src/host/Linux-SGX/sgx_thread.c

@@ -3,8 +3,8 @@
 #include "sgx_internal.h"
 #include "pal_security.h"
 
-#include <pthread.h>
 #include <linux/futex.h>
+#include <linux/signal.h>
 #include <asm/errno.h>
 #include <asm/signal.h>
 #include <asm/prctl.h>
@@ -12,12 +12,8 @@
 #include "sgx_enclave.h"
 #include "debugger/sgx_gdb.h"
 
-__thread struct pal_enclave * current_enclave;
-__thread sgx_arch_tcs_t * current_tcs;
-
 struct thread_map {
     unsigned int     tid;
-    bool             created_by_pthread;
     sgx_arch_tcs_t * tcs;
 };
 
@@ -25,13 +21,28 @@ static sgx_arch_tcs_t * enclave_tcs;
 static int enclave_thread_num;
 static struct thread_map * enclave_thread_map;
 
-pthread_mutex_t tcs_lock = PTHREAD_MUTEX_INITIALIZER;
+static void spin_lock(struct atomic_int* p) {
+    while (atomic_cmpxchg(p, 0, 1)) {
+        while (atomic_read(p) == 1)
+            CPU_RELAX();
+    }
+}
+
+static void spin_unlock(struct atomic_int* p) {
+    atomic_set(p, 0);
+}
+
+static struct atomic_int tcs_lock = ATOMIC_INIT(0);
 
 void create_tcs_mapper (void * tcs_base, unsigned int thread_num)
 {
+    size_t thread_map_size = ALIGN_UP_POW2(sizeof(struct thread_map) * thread_num, PRESET_PAGESIZE);
+
     enclave_tcs = tcs_base;
-    enclave_thread_map = malloc(sizeof(struct thread_map) * thread_num);
     enclave_thread_num = thread_num;
+    enclave_thread_map = (struct thread_map*)INLINE_SYSCALL(mmap, 6, NULL, thread_map_size,
+                                                            PROT_READ | PROT_WRITE,
+                                                            MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
 
     for (uint32_t i = 0 ; i < thread_num ; i++) {
         enclave_thread_map[i].tid = 0;
@@ -39,68 +50,162 @@ void create_tcs_mapper (void * tcs_base, unsigned int thread_num)
     }
 }
 
-void map_tcs(unsigned int tid, bool created_by_pthread) {
-    pthread_mutex_lock(&tcs_lock);
+void map_tcs(unsigned int tid) {
+    spin_lock(&tcs_lock);
     for (int i = 0 ; i < enclave_thread_num ; i++)
         if (!enclave_thread_map[i].tid) {
             enclave_thread_map[i].tid = tid;
-            enclave_thread_map[i].created_by_pthread = created_by_pthread;
-            current_tcs = enclave_thread_map[i].tcs;
+            get_tcb_linux()->tcs = enclave_thread_map[i].tcs;
             ((struct enclave_dbginfo *) DBGINFO_ADDR)->thread_tids[i] = tid;
             break;
         }
-    pthread_mutex_unlock(&tcs_lock);
+    spin_unlock(&tcs_lock);
 }
 
-/* return true if unmapped thread was created with pthread_create(), false otherwise */
-bool unmap_tcs(void) {
-    int index = current_tcs - enclave_tcs;
-    bool ret = false;
+void unmap_tcs(void) {
+    int index = get_tcb_linux()->tcs - enclave_tcs;
     struct thread_map * map = &enclave_thread_map[index];
 
     assert(index < enclave_thread_num);
 
-    pthread_mutex_lock(&tcs_lock);
-    current_tcs = NULL;
+    spin_lock(&tcs_lock);
+    get_tcb_linux()->tcs = NULL;
     ((struct enclave_dbginfo *) DBGINFO_ADDR)->thread_tids[index] = 0;
     map->tid = 0;
-    ret = map->created_by_pthread;
-    pthread_mutex_unlock(&tcs_lock);
-
-    return ret;
+    spin_unlock(&tcs_lock);
 }
 
-static void * thread_start (void * arg)
-{
+/*
+ * 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
+ * of the thread. The rest of the TCB is used as the alternative stack for signal
+ * handling. Notice that this sets up the untrusted thread -- an enclave thread is set
+ * up by other means (e.g., the GS register is set by an SGX-enforced TCS.OGSBASGX).
+ */
+int pal_thread_init(void* tcbptr) {
+    PAL_TCB_LINUX* tcb = tcbptr;
+    int ret;
+
+    /* set GS reg of this thread to thread's TCB; after this point, can use get_tcb_linux() */
+    ret = INLINE_SYSCALL(arch_prctl, 2, ARCH_SET_GS, tcb);
+    if (IS_ERR(ret)) {
+        ret = -EPERM;
+        goto out;
+    }
+
+    if (tcb->alt_stack) {
+        /* align stack to 16 bytes */
+        void* alt_stack = ALIGN_DOWN_PTR(tcb, 16);
+        assert(alt_stack > tcb->alt_stack);
+        stack_t ss;
+        ss.ss_sp    = alt_stack;
+        ss.ss_flags = 0;
+        ss.ss_size  = alt_stack - tcb->alt_stack;
+
+        ret = INLINE_SYSCALL(sigaltstack, 2, &ss, NULL);
+        if (IS_ERR(ret)) {
+            ret = -EPERM;
+            goto out;
+        }
+    }
+
     int tid = INLINE_SYSCALL(gettid, 0);
-    map_tcs(tid, /*created_by_pthread=*/true);
-    current_enclave = arg;
+    map_tcs(tid);  /* updates tcb->tcs */
 
-    if (!current_tcs) {
+    if (!tcb->tcs) {
         SGX_DBG(DBG_E,
                 "There are no available TCS pages left for a new thread!\n"
                 "Please try to increase sgx.thread_num in the manifest.\n"
                 "The current value is %d\n", enclave_thread_num);
-        return NULL;
+        ret = -ENOMEM;
+        goto out;
     }
 
+    if (!tcb->stack) {
+        /* only first thread doesn't have a stack (it uses the one provided by Linux); first
+         * thread calls ecall_enclave_start() instead of ecall_thread_start() so just exit */
+        return 0;
+    }
+
+    /* not-first (child) thread, start it */
     ecall_thread_start();
+
     unmap_tcs();
-    return NULL;
+    ret = 0;
+out:
+    INLINE_SYSCALL(munmap, 2, tcb->stack, THREAD_STACK_SIZE + ALT_STACK_SIZE);
+    return ret;
 }
 
-void thread_exit(void* rv) {
-    pthread_exit(rv);
-}
+noreturn void thread_exit(int status) {
+    PAL_TCB_LINUX* tcb = get_tcb_linux();
 
-int clone_thread(void) {
-    pthread_t thread;
+    /* technically, async signals were already blocked before calling this function
+     * (by sgx_ocall_exit()) but we keep it here for future proof */
+    block_async_signals(true);
+
+    if (tcb->alt_stack) {
+        stack_t ss;
+        ss.ss_sp    = NULL;
+        ss.ss_flags = SS_DISABLE;
+        ss.ss_size  = 0;
 
-    pthread_attr_t attr;
-    pthread_attr_init(&attr);
-    pthread_attr_setdetachstate(&attr, PTHREAD_CREATE_DETACHED);
+        /* take precautions to unset the TCB and alternative stack first */
+        INLINE_SYSCALL(arch_prctl, 2, ARCH_SET_GS, 0);
+        INLINE_SYSCALL(sigaltstack, 2, &ss, NULL);
+    }
+
+    /* free the thread stack (via munmap) and exit; note that exit() needs a "status" arg
+     * but it could be allocated on a stack, so we must put it in register and do asm */
+    __asm__ volatile("syscall \n\t"            /* all args are already prepared, call munmap */
+                     "movq %%rdx, %%rax \n\t"  /* prepare for exit: rax = __NR_exit */
+                     "movq %%rbx, %%rdi \n\t"  /* prepare for exit: rdi = status    */
+                     "syscall \n\t"            /* all args are prepared, call exit  */
+                     : /* no output regs since we don't return from exit */
+                     : "a"(__NR_munmap), "D"(tcb->stack), "S"(THREAD_STACK_SIZE + ALT_STACK_SIZE),
+                       "d"(__NR_exit), "b"(status)
+                     : "cc", "rcx", "r11", "memory"  /* syscall instr clobbers cc, rcx, and r11 */
+    );
+
+    while (true) {
+        /* nothing */
+    }
+}
 
-    return pthread_create(&thread, &attr, thread_start, current_enclave);
+int clone_thread(void) {
+    int ret = 0;
+
+    void* stack = (void*)INLINE_SYSCALL(mmap, 6, NULL, THREAD_STACK_SIZE + ALT_STACK_SIZE,
+                                        PROT_READ | PROT_WRITE,
+                                        MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
+    if (IS_ERR_P(stack))
+        return -ENOMEM;
+
+    void* child_stack_top = stack + THREAD_STACK_SIZE;
+
+    /* initialize TCB at the top of the alternative stack */
+    PAL_TCB_LINUX* tcb = child_stack_top + ALT_STACK_SIZE - sizeof(PAL_TCB_LINUX);
+    tcb->common.self   = &tcb->common;
+    tcb->alt_stack     = child_stack_top;
+    tcb->stack         = stack;
+    tcb->tcs           = NULL;  /* initialized by child thread */
+
+    /* align child_stack to 16 */
+    child_stack_top = ALIGN_DOWN_PTR(child_stack_top, 16);
+
+    int dummy_parent_tid_field = 0;
+    ret = clone(pal_thread_init, child_stack_top,
+                CLONE_VM|CLONE_FS|CLONE_FILES|CLONE_SYSVSEM|
+                CLONE_THREAD|CLONE_SIGHAND|CLONE_PTRACE|
+                CLONE_PARENT_SETTID,
+                (void*) tcb,
+                &dummy_parent_tid_field, NULL);
+
+    if (IS_ERR(ret)) {
+        INLINE_SYSCALL(munmap, 2, stack, THREAD_STACK_SIZE + ALT_STACK_SIZE);
+        return -ERRNO(ret);
+    }
+    return 0;
 }
 
 int interrupt_thread (void * tcs)
@@ -111,6 +216,6 @@ int interrupt_thread (void * tcs)
         return -EINVAL;
     if (!map->tid)
         return -EINVAL;
-    INLINE_SYSCALL(tgkill, 3, PAL_SEC()->pid, map->tid, SIGCONT);
+    INLINE_SYSCALL(tgkill, 3, pal_enclave.pal_sec.pid, map->tid, SIGCONT);
     return 0;
 }

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

@@ -36,6 +36,20 @@ struct enclave_tls {
     };
 };
 
+typedef struct pal_tcb_linux {
+    PAL_TCB common;
+    struct {
+        /* private to untrusted Linux PAL, unique to each untrusted thread */
+        sgx_arch_tcs_t*     tcs;       /* TCS page of SGX corresponding to thread, for EENTER */
+        void*               stack;     /* bottom of stack, for later freeing when thread exits */
+        void*               alt_stack; /* bottom of alt stack, for child thread to init alt stack */
+    };
+} PAL_TCB_LINUX;
+
+static inline PAL_TCB_LINUX* get_tcb_linux(void) {
+    return (PAL_TCB_LINUX*)pal_get_tcb();
+}
+
 #ifndef DEBUG
 extern uint64_t dummy_debug_variable;
 #endif