Ignore:
File:
1 edited

Legend:

Unmodified
Added
Removed
  • libcfa/src/concurrency/kernel.cfa

    rf586539 r57c764c4  
    1010// Created On       : Tue Jan 17 12:27:26 2017
    1111// Last Modified By : Peter A. Buhr
    12 // Last Modified On : Tue Feb  4 13:03:15 2020
    13 // Update Count     : 58
     12// Last Modified On : Thu Jun 20 17:21:23 2019
     13// Update Count     : 25
    1414//
    1515
     
    2626#include <signal.h>
    2727#include <unistd.h>
    28 #include <limits.h>                                                                             // PTHREAD_STACK_MIN
    29 #include <sys/mman.h>                                                                   // mprotect
    3028}
    3129
     
    4240//-----------------------------------------------------------------------------
    4341// Some assembly required
    44 #if defined( __i386 )
     42#if   defined( __i386 )
    4543        #define CtxGet( ctx )        \
    4644                __asm__ volatile (     \
     
    110108//-----------------------------------------------------------------------------
    111109//Start and stop routine for the kernel, declared first to make sure they run first
    112 static void __kernel_startup (void) __attribute__(( constructor( STARTUP_PRIORITY_KERNEL ) ));
    113 static void __kernel_shutdown(void) __attribute__(( destructor ( STARTUP_PRIORITY_KERNEL ) ));
    114 
    115 //-----------------------------------------------------------------------------
    116 // Kernel Scheduling logic
    117 static $thread * __next_thread(cluster * this);
    118 static void __run_thread(processor * this, $thread * dst);
    119 static $thread * __halt(processor * this);
    120 static bool __wake_one(cluster * cltr, bool was_empty);
    121 static bool __wake_proc(processor *);
     110static void kernel_startup(void)  __attribute__(( constructor( STARTUP_PRIORITY_KERNEL ) ));
     111static void kernel_shutdown(void) __attribute__(( destructor ( STARTUP_PRIORITY_KERNEL ) ));
    122112
    123113//-----------------------------------------------------------------------------
     
    125115KERNEL_STORAGE(cluster,         mainCluster);
    126116KERNEL_STORAGE(processor,       mainProcessor);
    127 KERNEL_STORAGE($thread, mainThread);
     117KERNEL_STORAGE(thread_desc,     mainThread);
    128118KERNEL_STORAGE(__stack_t,       mainThreadCtx);
    129119
    130120cluster     * mainCluster;
    131121processor   * mainProcessor;
    132 $thread * mainThread;
     122thread_desc * mainThread;
    133123
    134124extern "C" {
    135         struct { __dllist_t(cluster) list; __spinlock_t lock; } __cfa_dbg_global_clusters;
     125struct { __dllist_t(cluster) list; __spinlock_t lock; } __cfa_dbg_global_clusters;
    136126}
    137127
     
    141131// Global state
    142132thread_local struct KernelThreadData kernelTLS __attribute__ ((tls_model ( "initial-exec" ))) = {
    143         NULL,                                                                                           // cannot use 0p
     133        NULL,
    144134        NULL,
    145135        { 1, false, false },
     
    150140// Struct to steal stack
    151141struct current_stack_info_t {
    152         __stack_t * storage;                                                            // pointer to stack object
    153         void * base;                                                                            // base of stack
    154         void * limit;                                                                           // stack grows towards stack limit
    155         void * context;                                                                         // address of cfa_context_t
     142        __stack_t * storage;            // pointer to stack object
     143        void *base;                             // base of stack
     144        void *limit;                    // stack grows towards stack limit
     145        void *context;                  // address of cfa_context_t
    156146};
    157147
     
    172162// Main thread construction
    173163
    174 void ?{}( $coroutine & this, current_stack_info_t * info) with( this ) {
     164void ?{}( coroutine_desc & this, current_stack_info_t * info) with( this ) {
    175165        stack.storage = info->storage;
    176166        with(*stack.storage) {
     
    182172        name = "Main Thread";
    183173        state = Start;
    184         starter = 0p;
    185         last = 0p;
    186         cancellation = 0p;
    187 }
    188 
    189 void ?{}( $thread & this, current_stack_info_t * info) with( this ) {
     174        starter = NULL;
     175        last = NULL;
     176        cancellation = NULL;
     177}
     178
     179void ?{}( thread_desc & this, current_stack_info_t * info) with( this ) {
    190180        state = Start;
    191181        self_cor{ info };
     
    195185        self_mon.recursion = 1;
    196186        self_mon_p = &self_mon;
    197         next = 0p;
    198 
    199         node.next = 0p;
    200         node.prev = 0p;
     187        next = NULL;
     188
     189        node.next = NULL;
     190        node.prev = NULL;
    201191        doregister(curr_cluster, this);
    202192
     
    216206}
    217207
    218 static void * __invoke_processor(void * arg);
    219 
    220 void ?{}(processor & this, const char name[], cluster & cltr) with( this ) {
     208static void start(processor * this);
     209void ?{}(processor & this, const char * name, cluster & cltr) with( this ) {
    221210        this.name = name;
    222211        this.cltr = &cltr;
    223212        terminated{ 0 };
    224         destroyer = 0p;
    225213        do_terminate = false;
    226         preemption_alarm = 0p;
     214        preemption_alarm = NULL;
    227215        pending_preemption = false;
    228216        runner.proc = &this;
    229217
    230         idle{};
    231 
    232         __cfaabi_dbg_print_safe("Kernel : Starting core %p\n", &this);
    233 
    234         this.stack = __create_pthread( &this.kernel_thread, __invoke_processor, (void *)&this );
    235 
    236         __cfaabi_dbg_print_safe("Kernel : core %p started\n", &this);
     218        idleLock{};
     219
     220        start( &this );
    237221}
    238222
     
    242226
    243227                __atomic_store_n(&do_terminate, true, __ATOMIC_RELAXED);
    244                 __wake_proc( &this );
     228                wake( &this );
    245229
    246230                P( terminated );
     
    248232        }
    249233
    250         pthread_join( kernel_thread, 0p );
    251         free( this.stack );
    252 }
    253 
    254 void ?{}(cluster & this, const char name[], Duration preemption_rate) with( this ) {
     234        pthread_join( kernel_thread, NULL );
     235}
     236
     237void ?{}(cluster & this, const char * name, Duration preemption_rate) with( this ) {
    255238        this.name = name;
    256239        this.preemption_rate = preemption_rate;
     
    272255// Kernel Scheduling logic
    273256//=============================================================================================
     257static void runThread(processor * this, thread_desc * dst);
     258static void finishRunning(processor * this);
     259static void halt(processor * this);
     260
    274261//Main of the processor contexts
    275262void main(processorCtx_t & runner) {
     
    291278                __cfaabi_dbg_print_safe("Kernel : core %p started\n", this);
    292279
    293                 $thread * readyThread = 0p;
    294                 for( unsigned int spin_count = 0; ! __atomic_load_n(&this->do_terminate, __ATOMIC_SEQ_CST); spin_count++ ) {
    295                         // Try to get the next thread
    296                         readyThread = __next_thread( this->cltr );
    297 
    298                         // If no ready thread
    299                         if( readyThread == 0p ) {
    300                                 // Block until a thread is ready
    301                                 readyThread = __halt(this);
     280                thread_desc * readyThread = NULL;
     281                for( unsigned int spin_count = 0; ! __atomic_load_n(&this->do_terminate, __ATOMIC_SEQ_CST); spin_count++ )
     282                {
     283                        readyThread = nextThread( this->cltr );
     284
     285                        if(readyThread)
     286                        {
     287                                verify( ! kernelTLS.preemption_state.enabled );
     288
     289                                runThread(this, readyThread);
     290
     291                                verify( ! kernelTLS.preemption_state.enabled );
     292
     293                                //Some actions need to be taken from the kernel
     294                                finishRunning(this);
     295
     296                                spin_count = 0;
    302297                        }
    303 
    304                         // Check if we actually found a thread
    305                         if( readyThread ) {
    306                                 /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    307                                 /* paranoid */ verifyf( readyThread->state == Ready || readyThread->preempted != __NO_PREEMPTION, "state : %d, preempted %d\n", readyThread->state, readyThread->preempted);
    308                                 /* paranoid */ verifyf( readyThread->next == 0p, "Expected null got %p", readyThread->next );
    309 
    310                                 // We found a thread run it
    311                                 __run_thread(this, readyThread);
    312 
    313                                 /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
     298                        else
     299                        {
     300                                // spin(this, &spin_count);
     301                                halt(this);
    314302                        }
    315303                }
     
    323311
    324312        __cfaabi_dbg_print_safe("Kernel : core %p terminated\n", this);
    325 
    326         // HACK : the coroutine context switch expects this_thread to be set
    327         // and it make sense for it to be set in all other cases except here
    328         // fake it
    329         if( this == mainProcessor ) kernelTLS.this_thread = mainThread;
    330313}
    331314
     
    336319// runThread runs a thread by context switching
    337320// from the processor coroutine to the target thread
    338 static void __run_thread(processor * this, $thread * thrd_dst) {
    339         $coroutine * proc_cor = get_coroutine(this->runner);
     321static void runThread(processor * this, thread_desc * thrd_dst) {
     322        coroutine_desc * proc_cor = get_coroutine(this->runner);
     323
     324        // Reset the terminating actions here
     325        this->finish.action_code = No_Action;
    340326
    341327        // Update global state
    342328        kernelTLS.this_thread = thrd_dst;
    343329
    344         // set state of processor coroutine to inactive
    345         verify(proc_cor->state == Active);
    346         proc_cor->state = Blocked;
    347 
    348         // Actually run the thread
    349         RUNNING:  while(true) {
    350                 if(unlikely(thrd_dst->preempted)) {
    351                         thrd_dst->preempted = __NO_PREEMPTION;
    352                         verify(thrd_dst->state == Active  || thrd_dst->state == Rerun);
    353                 } else {
    354                         verify(thrd_dst->state == Blocked || thrd_dst->state == Ready); // Ready means scheduled normally, blocked means rerun
    355                         thrd_dst->state = Active;
     330        // set state of processor coroutine to inactive and the thread to active
     331        proc_cor->state = proc_cor->state == Halted ? Halted : Inactive;
     332        thrd_dst->state = Active;
     333
     334        // set context switch to the thread that the processor is executing
     335        verify( thrd_dst->context.SP );
     336        CtxSwitch( &proc_cor->context, &thrd_dst->context );
     337        // when CtxSwitch returns we are back in the processor coroutine
     338
     339        // set state of processor coroutine to active and the thread to inactive
     340        thrd_dst->state = thrd_dst->state == Halted ? Halted : Inactive;
     341        proc_cor->state = Active;
     342}
     343
     344// KERNEL_ONLY
     345static void returnToKernel() {
     346        coroutine_desc * proc_cor = get_coroutine(kernelTLS.this_processor->runner);
     347        thread_desc * thrd_src = kernelTLS.this_thread;
     348
     349        // set state of current coroutine to inactive
     350        thrd_src->state = thrd_src->state == Halted ? Halted : Inactive;
     351        proc_cor->state = Active;
     352        int local_errno = *__volatile_errno();
     353        #if defined( __i386 ) || defined( __x86_64 )
     354                __x87_store;
     355        #endif
     356
     357        // set new coroutine that the processor is executing
     358        // and context switch to it
     359        verify( proc_cor->context.SP );
     360        CtxSwitch( &thrd_src->context, &proc_cor->context );
     361
     362        // set state of new coroutine to active
     363        proc_cor->state = proc_cor->state == Halted ? Halted : Inactive;
     364        thrd_src->state = Active;
     365
     366        #if defined( __i386 ) || defined( __x86_64 )
     367                __x87_load;
     368        #endif
     369        *__volatile_errno() = local_errno;
     370}
     371
     372// KERNEL_ONLY
     373// Once a thread has finished running, some of
     374// its final actions must be executed from the kernel
     375static void finishRunning(processor * this) with( this->finish ) {
     376        verify( ! kernelTLS.preemption_state.enabled );
     377        choose( action_code ) {
     378        case No_Action:
     379                break;
     380        case Release:
     381                unlock( *lock );
     382        case Schedule:
     383                ScheduleThread( thrd );
     384        case Release_Schedule:
     385                unlock( *lock );
     386                ScheduleThread( thrd );
     387        case Release_Multi:
     388                for(int i = 0; i < lock_count; i++) {
     389                        unlock( *locks[i] );
    356390                }
    357 
    358                 __cfaabi_dbg_debug_do(
    359                         thrd_dst->park_stale   = true;
    360                         thrd_dst->unpark_stale = true;
    361                 )
    362 
    363                 /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    364                 /* paranoid */ verifyf( ((uintptr_t)thrd_dst->context.SP) < ((uintptr_t)__get_stack(thrd_dst->curr_cor)->base ) || thrd_dst->curr_cor == proc_cor, "ERROR : Destination $thread %p has been corrupted.\n StackPointer too small.\n", thrd_dst ); // add escape condition if we are setting up the processor
    365                 /* paranoid */ verifyf( ((uintptr_t)thrd_dst->context.SP) > ((uintptr_t)__get_stack(thrd_dst->curr_cor)->limit) || thrd_dst->curr_cor == proc_cor, "ERROR : Destination $thread %p has been corrupted.\n StackPointer too large.\n", thrd_dst ); // add escape condition if we are setting up the processor
    366 
    367                 // set context switch to the thread that the processor is executing
    368                 verify( thrd_dst->context.SP );
    369                 __cfactx_switch( &proc_cor->context, &thrd_dst->context );
    370                 // when __cfactx_switch returns we are back in the processor coroutine
    371 
    372                 /* paranoid */ verifyf( ((uintptr_t)thrd_dst->context.SP) > ((uintptr_t)__get_stack(thrd_dst->curr_cor)->limit), "ERROR : Destination $thread %p has been corrupted.\n StackPointer too large.\n", thrd_dst );
    373                 /* paranoid */ verifyf( ((uintptr_t)thrd_dst->context.SP) < ((uintptr_t)__get_stack(thrd_dst->curr_cor)->base ), "ERROR : Destination $thread %p has been corrupted.\n StackPointer too small.\n", thrd_dst );
    374                 /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    375 
    376 
    377                 // We just finished running a thread, there are a few things that could have happened.
    378                 // 1 - Regular case : the thread has blocked and now one has scheduled it yet.
    379                 // 2 - Racy case    : the thread has blocked but someone has already tried to schedule it.
    380                 // 4 - Preempted
    381                 // In case 1, we may have won a race so we can't write to the state again.
    382                 // In case 2, we lost the race so we now own the thread.
    383 
    384                 if(unlikely(thrd_dst->preempted != __NO_PREEMPTION)) {
    385                         // The thread was preempted, reschedule it and reset the flag
    386                         __schedule_thread( thrd_dst );
    387                         break RUNNING;
     391        case Release_Multi_Schedule:
     392                for(int i = 0; i < lock_count; i++) {
     393                        unlock( *locks[i] );
    388394                }
    389 
    390                 // set state of processor coroutine to active and the thread to inactive
    391                 static_assert(sizeof(thrd_dst->state) == sizeof(int));
    392                 enum coroutine_state old_state = __atomic_exchange_n(&thrd_dst->state, Blocked, __ATOMIC_SEQ_CST);
    393                 __cfaabi_dbg_debug_do( thrd_dst->park_result = old_state; )
    394                 switch(old_state) {
    395                         case Halted:
    396                                 // The thread has halted, it should never be scheduled/run again, leave it back to Halted and move on
    397                                 thrd_dst->state = Halted;
    398 
    399                                 // We may need to wake someone up here since
    400                                 unpark( this->destroyer __cfaabi_dbg_ctx2 );
    401                                 this->destroyer = 0p;
    402                                 break RUNNING;
    403                         case Active:
    404                                 // This is case 1, the regular case, nothing more is needed
    405                                 break RUNNING;
    406                         case Rerun:
    407                                 // This is case 2, the racy case, someone tried to run this thread before it finished blocking
    408                                 // In this case, just run it again.
    409                                 continue RUNNING;
    410                         default:
    411                                 // This makes no sense, something is wrong abort
    412                                 abort("Finished running a thread that was Blocked/Start/Primed %d\n", old_state);
     395                for(int i = 0; i < thrd_count; i++) {
     396                        ScheduleThread( thrds[i] );
    413397                }
    414         }
    415 
    416         // Just before returning to the processor, set the processor coroutine to active
    417         proc_cor->state = Active;
    418         kernelTLS.this_thread = 0p;
    419 }
    420 
    421 // KERNEL_ONLY
    422 void returnToKernel() {
    423         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    424         $coroutine * proc_cor = get_coroutine(kernelTLS.this_processor->runner);
    425         $thread * thrd_src = kernelTLS.this_thread;
    426 
    427         // Run the thread on this processor
    428         {
    429                 int local_errno = *__volatile_errno();
    430                 #if defined( __i386 ) || defined( __x86_64 )
    431                         __x87_store;
    432                 #endif
    433                 verify( proc_cor->context.SP );
    434                 __cfactx_switch( &thrd_src->context, &proc_cor->context );
    435                 #if defined( __i386 ) || defined( __x86_64 )
    436                         __x87_load;
    437                 #endif
    438                 *__volatile_errno() = local_errno;
    439         }
    440 
    441         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    442         /* paranoid */ verifyf( ((uintptr_t)thrd_src->context.SP) < ((uintptr_t)__get_stack(thrd_src->curr_cor)->base ), "ERROR : Returning $thread %p has been corrupted.\n StackPointer too small.\n", thrd_src );
    443         /* paranoid */ verifyf( ((uintptr_t)thrd_src->context.SP) > ((uintptr_t)__get_stack(thrd_src->curr_cor)->limit), "ERROR : Returning $thread %p has been corrupted.\n StackPointer too large.\n", thrd_src );
     398        case Callback:
     399                callback();
     400        default:
     401                abort("KERNEL ERROR: Unexpected action to run after thread");
     402        }
    444403}
    445404
     
    448407// This is the entry point for processors (kernel threads)
    449408// It effectively constructs a coroutine by stealing the pthread stack
    450 static void * __invoke_processor(void * arg) {
     409static void * CtxInvokeProcessor(void * arg) {
    451410        processor * proc = (processor *) arg;
    452411        kernelTLS.this_processor = proc;
    453         kernelTLS.this_thread    = 0p;
     412        kernelTLS.this_thread    = NULL;
    454413        kernelTLS.preemption_state.[enabled, disable_count] = [false, 1];
    455414        // SKULLDUGGERY: We want to create a context for the processor coroutine
     
    464423
    465424        //Set global state
    466         kernelTLS.this_thread = 0p;
     425        kernelTLS.this_thread    = NULL;
    467426
    468427        //We now have a proper context from which to schedule threads
     
    480439        __cfaabi_dbg_print_safe("Kernel : core %p main ended (%p)\n", proc, &proc->runner);
    481440
    482         return 0p;
    483 }
    484 
    485 static void Abort( int ret, const char func[] ) {
    486         if ( ret ) {                                                                            // pthread routines return errno values
    487                 abort( "%s : internal error, error(%d) %s.", func, ret, strerror( ret ) );
    488         } // if
    489 } // Abort
    490 
    491 void * __create_pthread( pthread_t * pthread, void * (*start)(void *), void * arg ) {
    492         pthread_attr_t attr;
    493 
    494         Abort( pthread_attr_init( &attr ), "pthread_attr_init" ); // initialize attribute
    495 
    496         size_t stacksize;
    497         // default stack size, normally defined by shell limit
    498         Abort( pthread_attr_getstacksize( &attr, &stacksize ), "pthread_attr_getstacksize" );
    499         assert( stacksize >= PTHREAD_STACK_MIN );
    500 
    501         void * stack;
    502         __cfaabi_dbg_debug_do(
    503                 stack = memalign( __page_size, stacksize + __page_size );
    504                 // pthread has no mechanism to create the guard page in user supplied stack.
    505                 if ( mprotect( stack, __page_size, PROT_NONE ) == -1 ) {
    506                         abort( "mprotect : internal error, mprotect failure, error(%d) %s.", errno, strerror( errno ) );
    507                 } // if
    508         );
    509         __cfaabi_dbg_no_debug_do(
    510                 stack = malloc( stacksize );
    511         );
    512 
    513         Abort( pthread_attr_setstack( &attr, stack, stacksize ), "pthread_attr_setstack" );
    514 
    515         Abort( pthread_create( pthread, &attr, start, arg ), "pthread_create" );
    516         return stack;
     441        return NULL;
     442}
     443
     444static void start(processor * this) {
     445        __cfaabi_dbg_print_safe("Kernel : Starting core %p\n", this);
     446
     447        pthread_create( &this->kernel_thread, NULL, CtxInvokeProcessor, (void*)this );
     448
     449        __cfaabi_dbg_print_safe("Kernel : core %p started\n", this);
    517450}
    518451
    519452// KERNEL_ONLY
    520 static void __kernel_first_resume( processor * this ) {
    521         $thread * src = mainThread;
    522         $coroutine * dst = get_coroutine(this->runner);
    523 
    524         verify( ! kernelTLS.preemption_state.enabled );
    525 
    526         kernelTLS.this_thread->curr_cor = dst;
     453void kernel_first_resume( processor * this ) {
     454        thread_desc * src = mainThread;
     455        coroutine_desc * dst = get_coroutine(this->runner);
     456
     457        verify( ! kernelTLS.preemption_state.enabled );
     458
    527459        __stack_prepare( &dst->stack, 65000 );
    528         __cfactx_start(main, dst, this->runner, __cfactx_invoke_coroutine);
     460        CtxStart(&this->runner, CtxInvokeCoroutine);
    529461
    530462        verify( ! kernelTLS.preemption_state.enabled );
     
    533465        dst->starter = dst->starter ? dst->starter : &src->self_cor;
    534466
    535         // make sure the current state is still correct
    536         /* paranoid */ verify(src->state == Ready);
     467        // set state of current coroutine to inactive
     468        src->state = src->state == Halted ? Halted : Inactive;
    537469
    538470        // context switch to specified coroutine
    539471        verify( dst->context.SP );
    540         __cfactx_switch( &src->context, &dst->context );
    541         // when __cfactx_switch returns we are back in the src coroutine
    542 
    543         mainThread->curr_cor = &mainThread->self_cor;
    544 
    545         // make sure the current state has been update
    546         /* paranoid */ verify(src->state == Active);
     472        CtxSwitch( &src->context, &dst->context );
     473        // when CtxSwitch returns we are back in the src coroutine
     474
     475        // set state of new coroutine to active
     476        src->state = Active;
    547477
    548478        verify( ! kernelTLS.preemption_state.enabled );
     
    550480
    551481// KERNEL_ONLY
    552 static void __kernel_last_resume( processor * this ) {
    553         $coroutine * src = &mainThread->self_cor;
    554         $coroutine * dst = get_coroutine(this->runner);
     482void kernel_last_resume( processor * this ) {
     483        coroutine_desc * src = &mainThread->self_cor;
     484        coroutine_desc * dst = get_coroutine(this->runner);
    555485
    556486        verify( ! kernelTLS.preemption_state.enabled );
     
    558488        verify( dst->context.SP );
    559489
    560         // SKULLDUGGERY in debug the processors check that the
    561         // stack is still within the limit of the stack limits after running a thread.
    562         // that check doesn't make sense if we context switch to the processor using the
    563         // coroutine semantics. Since this is a special case, use the current context
    564         // info to populate these fields.
    565         __cfaabi_dbg_debug_do(
    566                 __stack_context_t ctx;
    567                 CtxGet( ctx );
    568                 mainThread->context.SP = ctx.SP;
    569                 mainThread->context.FP = ctx.FP;
    570         )
    571 
    572490        // context switch to the processor
    573         __cfactx_switch( &src->context, &dst->context );
     491        CtxSwitch( &src->context, &dst->context );
    574492}
    575493
    576494//-----------------------------------------------------------------------------
    577495// Scheduler routines
     496
    578497// KERNEL ONLY
    579 void __schedule_thread( $thread * thrd ) with( *thrd->curr_cluster ) {
    580         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    581         /* paranoid */ #if defined( __CFA_WITH_VERIFY__ )
    582         /* paranoid */ if( thrd->state == Blocked || thrd->state == Start ) assertf( thrd->preempted == __NO_PREEMPTION,
    583                           "Error inactive thread marked as preempted, state %d, preemption %d\n", thrd->state, thrd->preempted );
    584         /* paranoid */ if( thrd->preempted != __NO_PREEMPTION ) assertf(thrd->state == Active || thrd->state == Rerun,
    585                           "Error preempted thread marked as not currently running, state %d, preemption %d\n", thrd->state, thrd->preempted );
    586         /* paranoid */ #endif
    587         /* paranoid */ verifyf( thrd->next == 0p, "Expected null got %p", thrd->next );
    588 
    589         if (thrd->preempted == __NO_PREEMPTION) thrd->state = Ready;
    590 
    591         lock  ( ready_queue_lock __cfaabi_dbg_ctx2 );
    592         bool was_empty = !(ready_queue != 0);
    593         append( ready_queue, thrd );
     498void ScheduleThread( thread_desc * thrd ) {
     499        verify( thrd );
     500        verify( thrd->state != Halted );
     501
     502        verify( ! kernelTLS.preemption_state.enabled );
     503
     504        verifyf( thrd->next == NULL, "Expected null got %p", thrd->next );
     505
     506        with( *thrd->curr_cluster ) {
     507                lock  ( ready_queue_lock __cfaabi_dbg_ctx2 );
     508                bool was_empty = !(ready_queue != 0);
     509                append( ready_queue, thrd );
     510                unlock( ready_queue_lock );
     511
     512                if(was_empty) {
     513                        lock      (proc_list_lock __cfaabi_dbg_ctx2);
     514                        if(idles) {
     515                                wake_fast(idles.head);
     516                        }
     517                        unlock    (proc_list_lock);
     518                }
     519                else if( struct processor * idle = idles.head ) {
     520                        wake_fast(idle);
     521                }
     522
     523        }
     524
     525        verify( ! kernelTLS.preemption_state.enabled );
     526}
     527
     528// KERNEL ONLY
     529thread_desc * nextThread(cluster * this) with( *this ) {
     530        verify( ! kernelTLS.preemption_state.enabled );
     531        lock( ready_queue_lock __cfaabi_dbg_ctx2 );
     532        thread_desc * head = pop_head( ready_queue );
    594533        unlock( ready_queue_lock );
    595 
    596         __wake_one(thrd->curr_cluster, was_empty);
    597 
    598         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
     534        verify( ! kernelTLS.preemption_state.enabled );
     535        return head;
     536}
     537
     538void BlockInternal() {
     539        disable_interrupts();
     540        verify( ! kernelTLS.preemption_state.enabled );
     541        returnToKernel();
     542        verify( ! kernelTLS.preemption_state.enabled );
     543        enable_interrupts( __cfaabi_dbg_ctx );
     544}
     545
     546void BlockInternal( __spinlock_t * lock ) {
     547        disable_interrupts();
     548        with( *kernelTLS.this_processor ) {
     549                finish.action_code = Release;
     550                finish.lock        = lock;
     551        }
     552
     553        verify( ! kernelTLS.preemption_state.enabled );
     554        returnToKernel();
     555        verify( ! kernelTLS.preemption_state.enabled );
     556
     557        enable_interrupts( __cfaabi_dbg_ctx );
     558}
     559
     560void BlockInternal( thread_desc * thrd ) {
     561        disable_interrupts();
     562        with( * kernelTLS.this_processor ) {
     563                finish.action_code = Schedule;
     564                finish.thrd        = thrd;
     565        }
     566
     567        verify( ! kernelTLS.preemption_state.enabled );
     568        returnToKernel();
     569        verify( ! kernelTLS.preemption_state.enabled );
     570
     571        enable_interrupts( __cfaabi_dbg_ctx );
     572}
     573
     574void BlockInternal( __spinlock_t * lock, thread_desc * thrd ) {
     575        assert(thrd);
     576        disable_interrupts();
     577        with( * kernelTLS.this_processor ) {
     578                finish.action_code = Release_Schedule;
     579                finish.lock        = lock;
     580                finish.thrd        = thrd;
     581        }
     582
     583        verify( ! kernelTLS.preemption_state.enabled );
     584        returnToKernel();
     585        verify( ! kernelTLS.preemption_state.enabled );
     586
     587        enable_interrupts( __cfaabi_dbg_ctx );
     588}
     589
     590void BlockInternal(__spinlock_t * locks [], unsigned short count) {
     591        disable_interrupts();
     592        with( * kernelTLS.this_processor ) {
     593                finish.action_code = Release_Multi;
     594                finish.locks       = locks;
     595                finish.lock_count  = count;
     596        }
     597
     598        verify( ! kernelTLS.preemption_state.enabled );
     599        returnToKernel();
     600        verify( ! kernelTLS.preemption_state.enabled );
     601
     602        enable_interrupts( __cfaabi_dbg_ctx );
     603}
     604
     605void BlockInternal(__spinlock_t * locks [], unsigned short lock_count, thread_desc * thrds [], unsigned short thrd_count) {
     606        disable_interrupts();
     607        with( *kernelTLS.this_processor ) {
     608                finish.action_code = Release_Multi_Schedule;
     609                finish.locks       = locks;
     610                finish.lock_count  = lock_count;
     611                finish.thrds       = thrds;
     612                finish.thrd_count  = thrd_count;
     613        }
     614
     615        verify( ! kernelTLS.preemption_state.enabled );
     616        returnToKernel();
     617        verify( ! kernelTLS.preemption_state.enabled );
     618
     619        enable_interrupts( __cfaabi_dbg_ctx );
     620}
     621
     622void BlockInternal(__finish_callback_fptr_t callback) {
     623        disable_interrupts();
     624        with( *kernelTLS.this_processor ) {
     625                finish.action_code = Callback;
     626                finish.callback    = callback;
     627        }
     628
     629        verify( ! kernelTLS.preemption_state.enabled );
     630        returnToKernel();
     631        verify( ! kernelTLS.preemption_state.enabled );
     632
     633        enable_interrupts( __cfaabi_dbg_ctx );
    599634}
    600635
    601636// KERNEL ONLY
    602 static $thread * __next_thread(cluster * this) with( *this ) {
    603         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    604 
    605         lock( ready_queue_lock __cfaabi_dbg_ctx2 );
    606         $thread * head = pop_head( ready_queue );
    607         unlock( ready_queue_lock );
    608 
    609         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    610         return head;
    611 }
    612 
    613 void unpark( $thread * thrd __cfaabi_dbg_ctx_param2 ) {
    614         if( !thrd ) return;
    615 
    616         disable_interrupts();
    617         static_assert(sizeof(thrd->state) == sizeof(int));
    618 
    619         // record activity
    620         __cfaabi_dbg_record_thrd( *thrd, false, caller );
    621 
    622         enum coroutine_state old_state = __atomic_exchange_n(&thrd->state, Rerun, __ATOMIC_SEQ_CST);
    623         __cfaabi_dbg_debug_do( thrd->unpark_result = old_state; )
    624         switch(old_state) {
    625                 case Active:
    626                         // Wake won the race, the thread will reschedule/rerun itself
    627                         break;
    628                 case Blocked:
    629                         /* paranoid */ verify( ! thrd->preempted != __NO_PREEMPTION );
    630 
    631                         // Wake lost the race,
    632                         thrd->state = Blocked;
    633                         __schedule_thread( thrd );
    634                         break;
    635                 case Rerun:
    636                         abort("More than one thread attempted to schedule thread %p\n", thrd);
    637                         break;
    638                 case Halted:
    639                 case Start:
    640                 case Primed:
    641                 default:
    642                         // This makes no sense, something is wrong abort
    643                         abort();
    644         }
    645         enable_interrupts( __cfaabi_dbg_ctx );
    646 }
    647 
    648 void park( __cfaabi_dbg_ctx_param ) {
    649         /* paranoid */ verify( kernelTLS.preemption_state.enabled );
    650         disable_interrupts();
    651         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    652         /* paranoid */ verify( kernelTLS.this_thread->preempted == __NO_PREEMPTION );
    653 
    654         // record activity
    655         __cfaabi_dbg_record_thrd( *kernelTLS.this_thread, true, caller );
     637void LeaveThread(__spinlock_t * lock, thread_desc * thrd) {
     638        verify( ! kernelTLS.preemption_state.enabled );
     639        with( * kernelTLS.this_processor ) {
     640                finish.action_code = thrd ? Release_Schedule : Release;
     641                finish.lock        = lock;
     642                finish.thrd        = thrd;
     643        }
    656644
    657645        returnToKernel();
    658 
    659         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    660         enable_interrupts( __cfaabi_dbg_ctx );
    661         /* paranoid */ verify( kernelTLS.preemption_state.enabled );
    662 
    663 }
    664 
    665 // KERNEL ONLY
    666 void __leave_thread() {
    667         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    668         returnToKernel();
    669         abort();
    670 }
    671 
    672 // KERNEL ONLY
    673 bool force_yield( __Preemption_Reason reason ) {
    674         /* paranoid */ verify( kernelTLS.preemption_state.enabled );
    675         disable_interrupts();
    676         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    677 
    678         $thread * thrd = kernelTLS.this_thread;
    679         /* paranoid */ verify(thrd->state == Active || thrd->state == Rerun);
    680 
    681         // SKULLDUGGERY: It is possible that we are preempting this thread just before
    682         // it was going to park itself. If that is the case and it is already using the
    683         // intrusive fields then we can't use them to preempt the thread
    684         // If that is the case, abandon the preemption.
    685         bool preempted = false;
    686         if(thrd->next == 0p) {
    687                 preempted = true;
    688                 thrd->preempted = reason;
    689                 returnToKernel();
    690         }
    691 
    692         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
    693         enable_interrupts_noPoll();
    694         /* paranoid */ verify( kernelTLS.preemption_state.enabled );
    695 
    696         return preempted;
    697646}
    698647
     
    702651//-----------------------------------------------------------------------------
    703652// Kernel boot procedures
    704 static void __kernel_startup(void) {
     653static void kernel_startup(void) {
    705654        verify( ! kernelTLS.preemption_state.enabled );
    706655        __cfaabi_dbg_print_safe("Kernel : Starting\n");
     
    720669        // SKULLDUGGERY: the mainThread steals the process main thread
    721670        // which will then be scheduled by the mainProcessor normally
    722         mainThread = ($thread *)&storage_mainThread;
     671        mainThread = (thread_desc *)&storage_mainThread;
    723672        current_stack_info_t info;
    724673        info.storage = (__stack_t*)&storage_mainThreadCtx;
     
    732681        void ?{}(processorCtx_t & this, processor * proc) {
    733682                (this.__cor){ "Processor" };
    734                 this.__cor.starter = 0p;
     683                this.__cor.starter = NULL;
    735684                this.proc = proc;
    736685        }
     
    741690                terminated{ 0 };
    742691                do_terminate = false;
    743                 preemption_alarm = 0p;
     692                preemption_alarm = NULL;
    744693                pending_preemption = false;
    745694                kernel_thread = pthread_self();
     
    763712        // Add the main thread to the ready queue
    764713        // once resume is called on mainProcessor->runner the mainThread needs to be scheduled like any normal thread
    765         __schedule_thread(mainThread);
     714        ScheduleThread(mainThread);
    766715
    767716        // SKULLDUGGERY: Force a context switch to the main processor to set the main thread's context to the current UNIX
    768         // context. Hence, the main thread does not begin through __cfactx_invoke_thread, like all other threads. The trick here is that
     717        // context. Hence, the main thread does not begin through CtxInvokeThread, like all other threads. The trick here is that
    769718        // mainThread is on the ready queue when this call is made.
    770         __kernel_first_resume( kernelTLS.this_processor );
     719        kernel_first_resume( kernelTLS.this_processor );
    771720
    772721
     
    780729}
    781730
    782 static void __kernel_shutdown(void) {
     731static void kernel_shutdown(void) {
    783732        __cfaabi_dbg_print_safe("\n--------------------------------------------------\nKernel : Shutting down\n");
    784733
    785         /* paranoid */ verify( TL_GET( preemption_state.enabled ) );
     734        verify( TL_GET( preemption_state.enabled ) );
    786735        disable_interrupts();
    787         /* paranoid */ verify( ! kernelTLS.preemption_state.enabled );
     736        verify( ! kernelTLS.preemption_state.enabled );
    788737
    789738        // SKULLDUGGERY: Notify the mainProcessor it needs to terminates.
     
    791740        // which is currently here
    792741        __atomic_store_n(&mainProcessor->do_terminate, true, __ATOMIC_RELEASE);
    793         __kernel_last_resume( kernelTLS.this_processor );
     742        kernel_last_resume( kernelTLS.this_processor );
    794743        mainThread->self_cor.state = Halted;
    795744
     
    801750        // Destroy the main processor and its context in reverse order of construction
    802751        // These were manually constructed so we need manually destroy them
    803         ^(*mainProcessor){};
     752        ^(mainProcessor->runner){};
     753        ^(mainProcessor){};
    804754
    805755        // Final step, destroy the main thread since it is no longer needed
    806756        // Since we provided a stack to this taxk it will not destroy anything
    807         /* paranoid */ verify(mainThread->self_cor.stack.storage == (__stack_t*)(((uintptr_t)&storage_mainThreadCtx)| 0x1));
    808         ^(*mainThread){};
     757        ^(mainThread){};
    809758
    810759        ^(__cfa_dbg_global_clusters.list){};
     
    815764
    816765//=============================================================================================
    817 // Kernel Idle Sleep
     766// Kernel Quiescing
    818767//=============================================================================================
    819 static $thread * __halt(processor * this) with( *this ) {
    820         if( do_terminate ) return 0p;
    821 
    822         // First, lock the cluster idle
    823         lock( cltr->idle_lock __cfaabi_dbg_ctx2 );
    824 
    825         // Check if we can find a thread
    826         if( $thread * found = __next_thread( cltr ) ) {
    827                 unlock( cltr->idle_lock );
    828                 return found;
    829         }
    830 
    831         // Move this processor from the active list to the idle list
    832         move_to_front(cltr->procs, cltr->idles, *this);
    833 
    834         // Unlock the idle lock so we don't go to sleep with a lock
    835         unlock    (cltr->idle_lock);
    836 
    837         // We are ready to sleep
     768static void halt(processor * this) with( *this ) {
     769        // verify( ! __atomic_load_n(&do_terminate, __ATOMIC_SEQ_CST) );
     770
     771        with( *cltr ) {
     772                lock      (proc_list_lock __cfaabi_dbg_ctx2);
     773                remove    (procs, *this);
     774                push_front(idles, *this);
     775                unlock    (proc_list_lock);
     776        }
     777
    838778        __cfaabi_dbg_print_safe("Kernel : Processor %p ready to sleep\n", this);
    839         wait( idle );
    840 
    841         // We have woken up
     779
     780        wait( idleLock );
     781
    842782        __cfaabi_dbg_print_safe("Kernel : Processor %p woke up and ready to run\n", this);
    843783
    844         // Get ourself off the idle list
    845784        with( *cltr ) {
    846                 lock  (idle_lock __cfaabi_dbg_ctx2);
    847                 move_to_front(idles, procs, *this);
    848                 unlock(idle_lock);
    849         }
    850 
    851         // Don't check the ready queue again, we may not be in a position to run a thread
    852         return 0p;
    853 }
    854 
    855 // Wake a thread from the front if there are any
    856 static bool __wake_one(cluster * this, __attribute__((unused)) bool force) {
    857         // if we don't want to force check if we know it's false
    858         if( !this->idles.head && !force ) return false;
    859 
    860         // First, lock the cluster idle
    861         lock( this->idle_lock __cfaabi_dbg_ctx2 );
    862 
    863         // Check if there is someone to wake up
    864         if( !this->idles.head ) {
    865                 // Nope unlock and return false
    866                 unlock( this->idle_lock );
    867                 return false;
    868         }
    869 
    870         // Wake them up
    871         post( this->idles.head->idle );
    872 
    873         // Unlock and return true
    874         unlock( this->idle_lock );
    875         return true;
    876 }
    877 
    878 // Unconditionnaly wake a thread
    879 static bool __wake_proc(processor * this) {
    880         return post( this->idle );
     785                lock      (proc_list_lock __cfaabi_dbg_ctx2);
     786                remove    (idles, *this);
     787                push_front(procs, *this);
     788                unlock    (proc_list_lock);
     789        }
    881790}
    882791
     
    899808                sigemptyset( &mask );
    900809                sigaddset( &mask, SIGALRM );            // block SIGALRM signals
    901                 sigaddset( &mask, SIGUSR1 );            // block SIGALRM signals
    902                 sigsuspend( &mask );                            // block the processor to prevent further damage during abort
    903                 _exit( EXIT_FAILURE );                          // if processor unblocks before it is killed, terminate it
     810                sigsuspend( &mask );                    // block the processor to prevent further damage during abort
     811                _exit( EXIT_FAILURE );                  // if processor unblocks before it is killed, terminate it
    904812        }
    905813        else {
     
    912820
    913821void kernel_abort_msg( void * kernel_data, char * abort_text, int abort_text_size ) {
    914         $thread * thrd = kernel_data;
     822        thread_desc * thrd = kernel_data;
    915823
    916824        if(thrd) {
    917825                int len = snprintf( abort_text, abort_text_size, "Error occurred while executing thread %.256s (%p)", thrd->self_cor.name, thrd );
    918                 __cfaabi_bits_write( STDERR_FILENO, abort_text, len );
     826                __cfaabi_dbg_bits_write( abort_text, len );
    919827
    920828                if ( &thrd->self_cor != thrd->curr_cor ) {
    921829                        len = snprintf( abort_text, abort_text_size, " in coroutine %.256s (%p).\n", thrd->curr_cor->name, thrd->curr_cor );
    922                         __cfaabi_bits_write( STDERR_FILENO, abort_text, len );
     830                        __cfaabi_dbg_bits_write( abort_text, len );
    923831                }
    924832                else {
    925                         __cfaabi_bits_write( STDERR_FILENO, ".\n", 2 );
     833                        __cfaabi_dbg_bits_write( ".\n", 2 );
    926834                }
    927835        }
    928836        else {
    929837                int len = snprintf( abort_text, abort_text_size, "Error occurred outside of any thread.\n" );
    930                 __cfaabi_bits_write( STDERR_FILENO, abort_text, len );
     838                __cfaabi_dbg_bits_write( abort_text, len );
    931839        }
    932840}
     
    939847
    940848extern "C" {
    941         void __cfaabi_bits_acquire() {
     849        void __cfaabi_dbg_bits_acquire() {
    942850                lock( kernel_debug_lock __cfaabi_dbg_ctx2 );
    943851        }
    944852
    945         void __cfaabi_bits_release() {
     853        void __cfaabi_dbg_bits_release() {
    946854                unlock( kernel_debug_lock );
    947855        }
     
    968876
    969877                // atomically release spin lock and block
    970                 unlock( lock );
    971                 park( __cfaabi_dbg_ctx );
     878                BlockInternal( &lock );
    972879        }
    973880        else {
     
    976883}
    977884
    978 bool V(semaphore & this) with( this ) {
    979         $thread * thrd = 0p;
     885void V(semaphore & this) with( this ) {
     886        thread_desc * thrd = NULL;
    980887        lock( lock __cfaabi_dbg_ctx2 );
    981888        count += 1;
     
    988895
    989896        // make new owner
    990         unpark( thrd __cfaabi_dbg_ctx2 );
    991 
    992         return thrd != 0p;
     897        WakeThread( thrd );
    993898}
    994899
     
    1007912}
    1008913
    1009 void doregister( cluster * cltr, $thread & thrd ) {
     914void doregister( cluster * cltr, thread_desc & thrd ) {
    1010915        lock      (cltr->thread_list_lock __cfaabi_dbg_ctx2);
    1011916        cltr->nthreads += 1;
     
    1014919}
    1015920
    1016 void unregister( cluster * cltr, $thread & thrd ) {
     921void unregister( cluster * cltr, thread_desc & thrd ) {
    1017922        lock  (cltr->thread_list_lock __cfaabi_dbg_ctx2);
    1018923        remove(cltr->threads, thrd );
     
    1022927
    1023928void doregister( cluster * cltr, processor * proc ) {
    1024         lock      (cltr->idle_lock __cfaabi_dbg_ctx2);
     929        lock      (cltr->proc_list_lock __cfaabi_dbg_ctx2);
    1025930        cltr->nprocessors += 1;
    1026931        push_front(cltr->procs, *proc);
    1027         unlock    (cltr->idle_lock);
     932        unlock    (cltr->proc_list_lock);
    1028933}
    1029934
    1030935void unregister( cluster * cltr, processor * proc ) {
    1031         lock  (cltr->idle_lock __cfaabi_dbg_ctx2);
     936        lock  (cltr->proc_list_lock __cfaabi_dbg_ctx2);
    1032937        remove(cltr->procs, *proc );
    1033938        cltr->nprocessors -= 1;
    1034         unlock(cltr->idle_lock);
     939        unlock(cltr->proc_list_lock);
    1035940}
    1036941
     
    1039944__cfaabi_dbg_debug_do(
    1040945        extern "C" {
    1041                 void __cfaabi_dbg_record_lock(__spinlock_t & this, const char prev_name[]) {
     946                void __cfaabi_dbg_record(__spinlock_t & this, const char * prev_name) {
    1042947                        this.prev_name = prev_name;
    1043948                        this.prev_thrd = kernelTLS.this_thread;
    1044949                }
    1045 
    1046                 void __cfaabi_dbg_record_thrd($thread & this, bool park, const char prev_name[]) {
    1047                         if(park) {
    1048                                 this.park_caller   = prev_name;
    1049                                 this.park_stale    = false;
    1050                         }
    1051                         else {
    1052                                 this.unpark_caller = prev_name;
    1053                                 this.unpark_stale  = false;
    1054                         }
    1055                 }
    1056950        }
    1057951)
     
    1059953//-----------------------------------------------------------------------------
    1060954// Debug
    1061 bool threading_enabled(void) __attribute__((const)) {
     955bool threading_enabled(void) {
    1062956        return true;
    1063957}
Note: See TracChangeset for help on using the changeset viewer.