<<

ImplementingLightweight Threads D. Stein,D. Shah- SunSoft,Inc. ABSTRACT We describean implementationof a threadslibrary that providesextremely lightweight threads within a single uNrx processwhile allowing fully concurrentaccess to system resources.The threadsare lighnveightenough so that they can be createdquickly, there can be thousandspresent, and synchronizationcan be accomplishedrapidly. These goals are achievedby providinguser threads which multiplexon a pool of ketnel-supportedthreads of control. This pool is managedby the library and will automaticallygrow or shrink as requiredto ensurethat the processwill makeprogress while not usingan excessiveamount of kernel resources.The programmercan also tune the relationshipbetween threads and kernel supportedthreads of control.This paperfocuses on schedulingand synchronizinguser threads,and their interactionwith uNlx sigrralsin a multiplexingthreads library.

Introduction resourcesthat may be appliedto solvingthe problem at hand. This paper describesa threadslibrary imple- mentation that provides extremely lightweight Threadsshare the processinstructions and most threadswithin a singleuNIx processwhile allowing of its data.A changein shareddata by one fully concurrentaccess to system resources. The can be seen by the other threadsin the . threadsare lightweight enoughso that they can be Threads also share most of the createdquickly, there can be thousandspresent, and stateof a process.For example,if one threadopens synchronizationcan be accomplishedrapidly. These a file, another thread can read it. There is no goalsare achievedby providinguser threads which system-enforcedprotection between threads. multiplex on a pool of kernel-supportedthreads of There are a variety of synchronizationfacilities control. The implementationconsists of a threads to allow threads to cooperatein accessingshared library and a kernel that has been modified to sup- data.The synchronizationfacilities include mutual port multiple threadsof control within a singleuttx exclusion(mutex) locks, condition variables,sema- process[Eykholt 1992]. phores and multiple readers, single writer The paper containsseven sections:The first (readers/writer)locks. The synchronizationvariables sectionis an overviewof the threadsmodel exported are allocatedby the applicationin ordinarymemory. by the library. The secondsection is an overviewof Threadsin different processescan also synchronize the threads library architecture.Readers familiar with eachother via synchronizationvariables placed with the SunOSMulti+hread Architecture [Powell in sharedmemory or mappedfiles, even thoughthe 19911can skip the first two sections.The third sec- threadsin different processesare generallyinvisible tion details the interfacessupplied by the kernel to to eachother. Such synchronizationvariables must support multiple threadsof control within a single be markedas being process-sharedwhen they are process.The fourth section details how the threads initialized.Synchronization variables may also have library schedulesthreads on kernel-supportedthreads different variants that can, for example,have dif- of control. The fifth section details how the library ferent behavioreven though the samesyn- implementsthread synchronization.The sixth sec- chronization semantic (e.g., mutual exclusion) is tion details how the threadslibrary implementssig- maintained. nals even though threadsare not known to the ker- Each threadhas a programcounter and a stack nel. The final section briefly describesa way to to maintainof local variablesand returnaddresses. implementa debuggerthat understandsthreads. Each thread can make arbitrary system calls and interact with other processesin the usual ways. Threads Model Some operationsaffect all the threadsin a process. A traditionaluNx processhas a singlethread For example,if onethread calls exit ( ), all threads of control. In the SunOS Multi.thread Architecture aredestroyed. [Powell1991], there can be morethan one threadof Each threadhas its own signal mask.This per- control, or simply more threads, that execute mits a threadto block asynchronouslygenerated sig- independentlywithin a process. In general, the nals while it accessesstate that is also modifiedby a number or identities of threadsthat an application signal handler.Signals that are synchronouslygen- processapplies to a problem are invisible from out- erated (e.g., sIcsEGv) are sent to the thread that sidethe process.Threads can be viewedas execution causedthem. signals that are generatedexternally

Summer '92 USENIX - June 8-June 12, L992- San Antonio, TX ImplementingLightweight Threads D. Stein,D. Shah

are sent to one of the threadswithin a processthat independentpage faults, and may run in parallel on has it unmasked.If all threadsmask a signal, it is a multiprocessor.All the twps in the system are set pending on the processuntil a thread unmasks scheduledby the kernel onto the availableCpUs that signal. Signalscan also be sent to particular accordingto their schedulingclass and priority. threadswithin the same process. As in single- The threadslibrary threads on a pool threadedprocesses, the number of sigrrals received of lwps in the process,in much the sameway by the processis less than as the or equal to the number kernel schedulesLw?s pool sent. on a of processors3. Threadsthemselves are simply data structuresand All threadswithin a processshare the set of stacks that are maintainedby the library and are signalhandlersr. If the handleris set to SIc IGN unknownto the kernel.The threadlibrary can cause any receivedsignals are discarded.If the handTeris an LwP to stop executingits currentthread and start set to SIG_DFL the defaultaction (e.g., stop, ' con- executinga new threadwithout involving the operat- tinue, exit)ãpplies to the processas a *:nble. ing system.Similarly, threads may also be crèated, A processcan fork in one of two ways: The destroyed,blocked, âctivated, etc., without operating first way clones the entire processand all of its systeminvolvement. L\ryps are relatively much more threads.The secondway only reproducesthe calling expensivethan threadssince each one usesdedicated threadin the new process.This last is usefulwhen kernel resources.If the threadslibrary dedicatedan the processis simplygoing to call ( ). LwP to each threadas in [Cooper1990] then many applicationssuch as databases or window systems Threads Library Architecture could not use threadsfreely (e.g. one or more per client) Threads are the programmer's interface for or they would be inefficient. Although ihe window systemmay multi+hreading.Threads are implementedby a be best expressedas a large number dynamicallylinked library thar utilizes underlying of threads,only a few of the threadsevèr needto be active(i.e., kernel-supportedthread! of control, called light- requirekernel resources, other - thanvirtual memory) weight processes(rwrsf, as shownin Figure 1. at the sameinstant. Sometimesa particularthread must be visible = to the system.For example,when a threadmust sup- { Thread =rwn Q Q=cpu port real+ime response(e.g., mousetracking thread) and thus be scheduledwith respectto all the other executionentities in the system(i.e., global schedul- ing scope).The library accommodatesthis by allow- ing a thread to be permanentlybound to an lwp. Even when a threadis bound to an Lwp. it is still a thread and can interact or synchronizewith other threadsin the process,bound or unbound. By defining Kernel both levels of interface in the architecture,we make clear the distinctionbetween what the programmersees and what the kernel pro--and Hardware vides. Most programmersprogram using threads do not think aboutrwps. When it is appropriateto Figure 1: Multi-threadArchitecture Examples optimize the behaviorof the program,the program- mer hasthe ability to tune the relationshipbenveen Each lwp can be thought of as a virtual CpU threadsand Lwps. This allows programmersto struc- which is available for executing code or system turetheir applicationassuming extremely lightweight -of calls. Each Lwp is separarely dispatched by the ker- threadswhile bringing the appropriatedegree nel, may perform independent system calls, incur kernel-supportedconcurrency to bear on the compu- tation. A threads programmercan think of rwps used by the applicationas the degreeof real its own vector con- @ave of signal currencythat the applicationrequires. handlers was rejected becauseit would add a non-t¡ivial amount of' storage to each thread and it allowed the LIVP interfaces possibility of conflicting requestsby different threads(e.g., SIG_DFr/ and a handler) that would make both Lwps are like threads.They sharemost of the implementation and use difficult. In addition, per-thread processresources. Each rwp has a private set of canbe programmedon top of a sharedvector of handle¡s, registers and a signal mask. Lwps also have if zThe^required. Lwps in this document are fundamentally different JAchrally, thal th9 lwr library in SunOS4.0. Lack of imagination the SunOS5.0 kernelschedules kemel th¡eads and a desi¡e to conform to generally acceptedterminology on top of processors.Kemel th¡eads are used for the leadus to use the sâñe name. executioncontext underlying Lwrs. See[Eykholt 1992].

'92 Summer USENIX- June8-June 12,lgg} - SanAntonio, TX D. Stein,D. Shah Implementing Lightweight Threads attributesthat are unavailableto threads. For exam- initialization.When a Lwp synchronizationprimitive ple they have a kernel-supportedscheduling class, causesthe calling LwP to block, the lwp is then virtual time timers, an alternatesignal stack and a suspendedon a kernel-supportedsleep queue associ- profiling buffer. atedwith the offset in the mappedfile or the shared ó. The systeminterface to LwPs4is shownin Fig- memory segments This all-owslwps in different ure 2, processesto synchronizeeven though they have the synchronizationvariable mappedat different virtual int _hrp_create(, f1a9, lwpldp) addresses. vold _1w¡l_rnakecontext(ucp, func, arg, private, stack, size) i Both the ( and *_krp_getprivate( _Iwp_getprÍvate ) void ) t ( interfaces provide one vold _Iwp_eetprivate(ptr)t _Iwp_setprivate ) void _lwp_exlt( ) t pointerfsworth of storagethat is private to the Lwp. ittt _Iwp_wait(waltfor, departedp)i Typically,a threadspackage can usethis to point to twptd_t _lrdp_self( ) t int _lvrp_suBpênd(Iwptd)¡ its thread data structure.On SPARC this interface inÈ _krp_continuê(twpid), setsand getsthe contentsof, register Vog77. vold _Iwp_nut€x_lock(lwp_rûutexp)t void _1Ì¡p_!ìutox_unlock(hrp_nut€xli An alternativeto this approachis to have a lnt _1wp_cond_wait(Ierp_condp, hrp_nutexp)t private memory page per lwp. This requiresmore int _1wp_cond_tl¡ned\dait(lwp_condp, Iwp_nutêxp, perhaps tirneout ) i kerneleffort and more memoryrequirements int _Lwp_con't_eignal(hrp_condp)t for eachLwP. However,it is probablythe most rea- lnt _Iwp_cond_broadcast(hrp_condp)i sonablechoice on registerconstrained machines, or int _1wp_eena_walt(hrp_sernap)t lnt _Iwp_EernaJoBt ( let _Bemap) t on machineswhere user registershave not been ínt _Iwp_ki.I1(Iwpid, sig) t reservedto the system. Figure 2: lwP Interfaces The kernel also providestwo additionalsignals. The first, sIcLwP is simply a new signalreserved for The _Iwp_create ( ) interface creates threadspackages. It is used as an inter-Lwr signal another LwP within the process. The mechanismwhen directedto particularLwPs within _Iwp_makecontext ( ) interface creates a the processvia the _Iwp_kill( ) interface.The machinecontext that emulatesthe standardcalling secondsignal is sIctffAITINc.This is currently gen- sequenceto a function. The machine context can eratedby the kernelwhen it detectsthat all the Lwps then be passedto _lwp_create ( ). This allows in the processhave blocked in indefinitewaits. It is some degreeof machine independencewhen using used by threadspackages to ensurethat processes the LwP interfaces. don't deadlockindefinitely due to lack of execution The rwp synchronizationinterfaces implement resources. mutual exclusion locks, condition variables and ThreadsLibrary Implementation countingsemaphores. Variants are allowedthat can supportpriority inversionprevention protocols such Eachthread is representedby a threadstructure as priority inheritance.Counting semaphores are also that containsthe thread ID, an area to save the providedto ensurethat a synchronizationmechanism threadexecution context, the threadsignal masþ the that is safe with respectto asynchronoussignals is threadpriority, and a pointerto the threadstack. The available.In general,these routinesonly actually storagefor the stackis either automaticallyallocated enter the kernel when they have to. For example,in by the libraryor it is passedin by the applicationon some cases acquiring an LWP mutex does not threadcreation, Library allocatedstacks are obtained require kernel entry if there is no contention,The by mapping in pagesof anonymousmemory. The kernel has no knowledge of rwn synchronization library ensuresthat the page following the stack is variablesexcept during actualuse. invalid. This representsa "red zone" so that the processwill The l,wp synchronizationvariables are placed be signalledif a threadshould run off the stack.If the passed in memoryby the application.If they are placedin application in its own stack storage,it can provide a red zone pack sharedmemory or in mappedfiles that are accessible or the stacks in its own way. to other processes,they will synchronizeLwps betweenall the mappingprocesses even though the twps in different processesare generallyinvisible to @ call this the virtual address eachother. An advantageof usingmemory mapped andwhat mostpeople today call the virtualaddress (the files is that the synchronizationvariables along with effectiveaddresses computed by the instructions)was calledthe loeical address, other shareddata can be preservedin a file. The 6lntema[y]thekernel uses a hashqueue to synchronize applicationcan be restartedand continueusing its twrs. Thehash lookup is basedon a two wordvalue that shared synchronizationvariables without any representsthe internal file handle(vnode) and offset. /On SPARC,register 7097, is reservedto the systemin ffied hereare compatible with the ApplicationBinary Interface [USO 1990]. ABI thecunent UNIX Internationallwp interfaces. compliantapplications cannot use it.

Summer '92 USENIX - June 8,June L2, L992- San Antonio, TX Implementing Lightweight Threads D. Stein,D. Shah

When a thread is created, a thread ID is on any of the lwps in this pool. When a threadexe- assigned.In an earlier threadslibrary implementa- cutes,it is attachedto an Lwp and has all the attri- tion, the threadID was a pointer to the threadstruc- butesof being a kernel-supportedth¡ead. ture. This implementationwâs discardedin favor .index of All runnable,unbound, threads are on a user one where the threadID was used as an in a level, prioritized dispatch queue. Thread prioritiæ table of pointersto threadstructures. This allows the range from 0 to "infinity"e A thread's priority is library to give meaningfulenors when a threadID fixed in the sensethat the threadslibrarv does of an exited and deallocated passed not threadis to the changeit dynamically in Iibrary. as a time sharedscheduling discipline.It can be changedonly by the threaditself Threadlocal Storage or by another thread in the same process. The Threadshave some private storage(in addition unboundthread's priority is used only by the user to the stack)called thread-localstorage (TIS). Most level threadscheduler and is not known to the ker- variablesin the program are sharedamong all the nel. threads executingit, but each thread has its own copy of th¡ead-localvariables. C-onceptually, = thread- LwP [ = threadO = sYnch.variable local storageis unshared,statically allocateddata. It O is usedfor thread-privatedata that must be accessed LWP Pool quickly. The threadstructure itself and a versionof mi,ffol¡ z,srcwAITING errno that is private Poolerogth to the thread is allocatedin ¡* Run TLS. Thread-localstorage is obtainedvia a new #pragma, unshared, supportedby the compiler and linker. The contentsof TLS are zeroed,initiãlly; static initialization is not allowed. The size of TLS is computedby the run-time linker at programstart time by summing the sizes of the threadJocal storag_esegments of the linked libraries and comput- ing offsetsof the items in TLS8. The linker defines a symbol called _etls that representthe size of TLS. The threadslibrary usesthis at threadcreation time to allocatespace for TLS from the baseof the new thread'sstack. After programstartup, the size of Figure3: Thread TLS is fixed and can no longer grow. This restricts programmaticdynamic linking (i,e., dlopen( to )) Figure 3 gives an overview of the multiplexed libraries that do not contaínTLS. Becauseof these schedulingmechanism. An l,wp in the pool is either restrictions,thread-local storage is not an exported idling or running a thread.When an lwp is it interface. Instead a programmaticinterface ialled waits on a syncluonizationvariable (actually it can thread-specificdata 1992]is available. [POSIX use one of two, see below) for work to do. When a - On SPARC,global registerVogT is assumedby thread, T1 is made runnable, it is added to the the compiler to point to the base addressof TLS. dispatchqueue, and an idle lwp Ll (if one exists)in This is the same register used by the Lwp private the pool is awakenedby signalling the idle syn- stgrageinterfaces. The compiler generatescode for ch¡onizationvariable. Lwp Ll. wakesup and switches TLS referencesrelative to Vog7.The threadslibrary to the highestpriority threadon the dispatchqueue. ensuresthat VogTis set to the baseaddress of TLS If T1 blockson a local synchronizationobject (i.e., for the cunently executingthead. The impactof this one that is not sharedbetween processes)¡ Ll puts on threadscheduling is minimal. T1 on a sleepqueue and then switchesto the highest priority threadon the dispatchqueue. If the dispatch Thread Scheduling queueis empty, the Lwp goes back to idling. If all The threads library ímplements a thread l,wps in the pool are busy when T1 becomesn¡nn- schedulerthat multiplexesthread execution across a able,T1 simply stayson the dispatchqueue, waiting pool of nvps. The rwps in the pool are set up to be for an Lwp to becomeavailable. An t wp becomes nearly identical. This allows any threadto execute availableeither when a new one is addedto the pool or when one of the running th¡eads blocks on a process-localsynchronization variable, exits or stops, ffieß in TLS are computedat freeingits ¡,wp. staduptime ratherthan link time so that the amountof thread-localstorage required by any particularlibrary is gCurrently, not compiledinto the applicationbinary and the library "infinity" is the maximum number maychange it withoutbreaking the binary interface. representableby 32 bits.

'92 Summer USENIX- June8-June 12,lgg2 - SanAntonio, TX D. Stein, D. Shah Implementing Lightweight Threads

Thread States and the Two Level Model Whena boundthread blocks on a process-local An unbound thread can be in one of five dif- synchronizationvariable, its l,wp mustalso stop run- ferent states: RLJNNABII,, AcrIvE, SLEEPING, ning. It does so by waiting on a Lwp semaphore sroPPED, and zovnn. The transitions between these associatedwith the thread.The twp is now pørked. statesand the events that causethese transitionsare Whenthe boundthread unblocks, the parkingsema- shown in Figure 4. phore is signalledso that the LwP can continueexe- cutingthe tkead. wakeup When an unboundthread becomes blocked and there are no more RUNNABLEthreads, the Lwp that was running the thread also parlcs itself on the thread'ssemaphore, rather than idling on the idle stack and global condition variable. This optimizes the casewhere the blockedthread becomes runnable quickly, sinceit avoidsthe contextswitch to the idle stackand back to the thread. Threadscompete for Lwpsbased on their priori- ties just as kernel threadscompete for CPUs. A queueof AcTIvE threadsis maintained.If there is a possibility that a RUNNABLEthread has a higher priority than that of someAcTIvE thread, the AcTrvE Figure 4: Threadstates queue is searchedto find such a thread. If such a th¡eadis found, then this threadis removedfrom the While a threadis in the ecnvE state,its under- queue and preemptedfrom its lwp. This LwP then lying rwr can be runningon a processor,sleeping in schedulesanother thread which is typically the the kernel,stopped, or waitingfor a processoron the higher priority RUNNABLEthread which causedthe kernel'sdispatch queue. The four LwPstates and the preemption. transitionsbetween them can be lookedupon as a Thereare basically two caseswhen the needto detaileddescríption of the threadAcTIvE state. This preempt arises. One is when a newly RUNNABII, relationshipbetween LwP statesand threadstates is threadhas a higherpriority than that of the lowest shownin Figure5. priority AcTIvE thread and the other is when the priority of an AcTIvEthread is loweredbelow that of the highestpriority RUNNABLEthread. AcTIvEthreads are preemptedby setting a flag in the thread'sthread structure and then sendingits LwP a sIcLwP.The threadslibrary alwaysinstalls its own handlerfor slcLwP. When an LWpreceives the signal,the handlerchecks if the currentthread has a preemptionposted. If it is, it clearsthe flag andthen switchesto anotherthread. Oneside-effect of preemptionis that if the tar- get threadis blockedin the kernelexecuting a sys- tem call, it will be interruptedby slct w, and the will be re-startedwhen the thread Figure 5: LWP states resumesexecution after the preemption:This is only a problemif the systemcall shouldnot be re-started. Idling and parking TheSize of thetwn Pool Whenan unboundthread exits and there are no By default, the threadslibrary automatically more RUNNABLEthreads, the LwP that was running adjuststhe numberof ¡.wpsin the pool of lwps that the threadswitches to a small idle stackassociated areused to run unboundthreads. There are two main with eachLWP and idles by waiting on a global twr requirementsin settingthe size of this pool: First,it condition variable.When anotherthread becomes must not allow the programto deadlockdue to lack RUNNABLEthe global conditionvariable is signaled, of Lwps. For example if there are more runnable and an idling rwr wakesup and attemptsto run any unboundthreads than Lwps and all the active th¡eads RUNNABLEthreads. block in the kernel in indefinitewaits (e.g.,read a tty), then the processcan make no furtherprogress until one of the waiting threadsreturns. The second requirementis that the library should make efficient

Summer'92 USENIX- June8-June 12,1992 - SanAntonio, TX Implementing Lightweight Threads D. Stein,D. Shah

use of fwps. If the library were to createone Lwp The numberof rwps in the pool can grow to be for every thread, in many casesthese rwrs would greaterthan the numberof threadscurrently in the simply be idle and the operatingsystem would be processdue to previousreceptions of stcweitxo or overloaded by the resourcerequirements of many usesof thr_setconcurrency ( ). Thiscan result more LWpsthan are actuallybeing used.The advan- in an excessivenumber of lwrs in the pool. The tagesof a two-levelmodel would be lost. library therefore"ages" rwrs; they æe terminatedif I,et C be the total numberof ecrrvg andRUNTI- they are unsuedfor a "long" time, currently five enrE threadsin an application,at any instant. In minutes.This is implementedby setting a per-Lwp manycases, the valueof C will fluctuate,sometimes timer wheneverthe Lwp starts idling. If the timer quite dramaticallyas events come in and are pro- goesoff, the Lwp is terminated. cessed,or threadssynchronize. Ideally, if the time Mixed ScopeScheduling for creating or destroying an Lwp were zeroi the A mixture of bound and unboundth¡eads can optimal design would be for the threadslibrary to coexist in the same process. Applications such as keepthe size of the Lwp pool equalto C at all times. window systemscan benefit from such mixed scope This design would meet both requirementsfor the scheduling.A boundthread in the real-timeschedul- pool perfectly.In reality, since adjustingthe size of ing class can be devotedto fielding mouse events the Lwp pool has a cost, the threadslibrary doesnot which then take precedenceover all other unbound 'attemptto match it perfectlywith C sincethis would threads which are multiplexing over time-sharing be inefficient. L\,vPs.\len a boundthread calls the system,sprior- The current threads library implementation ity control interface,prÍocntl( ), it affects the startsby guaranteeingthat the applicationdoes not lwp it is bound to, and thus the th¡eaditself. Thus. deadlock.It does so by using the new sIcwAITINc bound,real-time threads can coexistwith unbouná signalthat is sentby the kernel when all the t wps in threads multiplexing across time-shared Lwps. the processblock in indefinitewaits. The threads Unboundthreads continue to be scheduledin a mul- library ignores srcwArrrNc by default. When the tiplexedfashion in the presenceof boundthreads. number of threadsexceeds the number of lwrs in Reapingboundl unbound threads the pool, the threadslibrary installsa handlerfor When a detachedthread sIcwAITrNG. If the threads library receives a þound or unbound) exits,it is put on a singlequeue, SIGWATTINGand there are runnableth¡eads, it creates called{eathrow and their stateis set to zoMBIp. a new Lwp and adds it to the pool. If there are no The action of free- ing a thread'sstack is runnablethreads at the time the signal is received not doneat threadexit time to minimize the cost andthe numberof threadsis lessthan the number of thread exit by deferring of unnecessary Lwpsin the pool, it disablessIGwAITINc. and expensivework. The threadslibrary has a specialthread called the reaperwhose job is to A more aggressive implementation might do this work periodically. The reaper runs when attempt to computea weighted time averageof the there are idle rwrs, or when a reap limit is reached application'sconcurrency requirements,and adjust (the deathrow gets full). The reapertraverses the the pool of twps more aggressively.Cunently, it is deathrow list, freeing the stack for each thread not known whether the advantagesof doing this that had beenusing a library allocatedstack. would outweighthe extra overheadof creatingmore LWPS. When an undetachedthread (bound or unbound) exits, it is addedto the zombie list. Threadson the The application sometimes knows its con- zombie list are reapedby the thread that executes currencyrequirements better thanthe threadslibrary. thr_join( ) on them. The threads library provides an interfacê, Sincethe reaperruns at high priority,it should thr_setconcurrency( ) that givesthe library a not be run too frequently.Yet, hint as to what the expectedconcurrency level is. it shouldnot be run too rarely, since For example,a multi-threadedfile copy program, the rate at which threadsare reaped hasan impacton the speedof thread with one input and one output threadmight sét its creation.This is becausethe reaper puts concunencylevel to two; a multi-threadedwindow the f¡eed stacks on a cache of available thread stacks, systemserver might set its concurrencylevel to the which speedsup stackallocation for new threads, number of expected simultaneouslyactive clients an otherwiseexpen- siveoperation. timesthe numberof threadsper client. If n is the level of concurrency given in Thread Synchrcnization thr_setconcurrency ( ), the threadslibrary will ensurethere are at leastn L\ilpsin the pool when the The threadslibrary implementstwo basic types-(ihe numberof threadsis greater than or equal to n. If of synchronization variables, process-local the number of threads is less than n, the library default) or process-shæed.In both casesthe libræv ensuresthat the numberof lwps is at least equal to ensuresthat the synchronizationprimitives them- the numberof threads.Any growth in the numberof selvesare safe with respectto asynchronoussignals LlryPspast n is due to receivinga SIGWAITING. and therefore they can be called from signal

'92 Summer USENIX - June 8-June lZ, LggZ- San Antonio. TX D. Stein,D. Shah ImplementingLightweight Threads handlers. overheadasync safe synchronizationprimitives are Process-localSynchronization Variables crucial for multi-threadedlibraries containing inter- nal critical sections,such as the threadslibræy and The default blocking behavior is to put the its clients,such as libc, etc. For example,consider a threadto sleep.Each synchronizationvariable has a call to the threadslibræy, say mutex_l.ock( sleep queueassociated with it. The synchronization ¡, which is intenupted asynchronouslywhile holding primitives put the blocking threadson the synchroni- an internalthreads library , .L. The asynchronous zationvariable's sleep queue and sunenders the exe- handlercould potentiallycall into mutex_lock( cuting thread to the scheduler. If the thread is ) also and try to acquire L again, resulting in a unbound,the schedulerdispatches another thread to deadlock.One way of making rnutex_locko the thread'sunderlying Lwp. A Bound thread,how- asyncsafe is to mask signalswhile in Z's critical ever, must stay permanentlybound to its l,wp so the section. Thus, efficient signal masking was an LIVP is parked on its thread importantgoal since it could provide efficient async Blocked threadsare awakenedwhen the syn- safecritical sections. ch¡onizationvariables become available. The syn- SignalModel Implementation chronizationprimitives that releaseblocked threads first checkif threadsare waiting for the synchroniza- Oneimplementation strategy would be for each tion væiables.A blockedthread is removedfrom the L\ryPthat is running a thread to reflect the thread's synchronizationvariable's sleep queue and is signal mask. This allows the kernel to directly dispatchedby the scheduler.If the threadis bound, choosea thread to signal from among the ectw the schedulerunparks the bound thread so that its threadswithin a process.The signalsthat a process lwp is dispatchedby the kernel, For unbound can receivechanges as the threadsin the application threads,the schedulersimply placesthe threadon a cycle throughthe AcïvE state. run queueconesponding to its priority. The threadis This strategyhas a problem with threadsthat then dispatchedto an Lwp in priority order. are rarely AcTIVE and are the only threadsin the Process-sharcdSynchronization variables applicationthat have certainsignals enabled. These threadsare essentiallyasleep waiting to be inter- Process-sharedsynchronization objects can also ruptedby a signalwhich they will neverreceive. In be placedin memorythat is accessibleto more than addition, a system call must be done wheneveran one processand can be used to synchronizethreads LwP switches between threads having different in different processes.Process-shared synchroniza- masksor when an activethread adjusts its mask. tion variablesmust be initialized when they are createdbecause their blocking behavioris different This problem can be solved if the lwr signal from the default. Each synchronizationprimitive masks and the AcrIvE thread signal masks are supportsan initializationfunction that must be called treatedmore independently. The set of signalsthat a to mark the process-sharedsynchronization variable processcan receiveis equalto the intersectionof all as process-shared.The primitives can then recognize the th¡eadsignal masks.The library ensuresthat the the synchronizationva¡iables that are shared and LWP signal mask is either equal to the threadmask provide the conect blocking behavior. The primi- or it is lessrestrictive. This meansoccasionally sig- tives rely on LwP synchronizationprimitives to put nals are sent by the kernel to AcTIVEthreads that the blocking th¡eads to sleep in the kernel still havethe signaldisabled. attachedto their Llvps, and to conectly synchronize When this occurs the threadslibrary prevents betweenprocesses. the signal from actually reaching the interrupted threadby interposingits own signalhandlers below Signals the application signal handlers.When a signal is The challengein providingthe SunOSMT sig- delivered, the global handler checks the current nal semanticsfor userthreads in a two-levelmodel thread'ssignal mask to determineif the threadcan of multi-threadingis that signalsare sent by the ker- receivethis signal.If the signalis masked,the glo- nel but user level threadsand their masksare invisi- bal handler sets the current LWP's signal mask to ble to the kernel.In particular,since signal delivery the cunentthread's signal mask. Then the signalis to a threadis dependenton the threadsignal mask, resentto the processif it is an undirectedsignal or the challenge is to elicit the correct program to its LWP if it is a directedsignal. The kernel sig- behavioreven though the kernel cannotmake the nal delivery mechanismprovides information that correctsignalling decisions because it cannotsee all allows the signal handler to distinguish between the masks. directedand undirectedsignals. If the signal is not masked,the global The implementationhas the additionalgoal of handlercalls the signal'sapplica- providing cheap async safe synchronizationprimi- tion handler.If the signal is not appropriatefor any global tives. A function is said to be asyncsafe if it is of the cunentlyACTIVE threads,the handler reentrantwith respectto signals,i.e., it is callable can causeone of the inactive threadsto run if it has from a signal handlerinvoked asynchronously.Low the signalunmasked.

Summer '92 USENIX - June 8-June12,1992 - SanAntonio, TX Implementing Lightweight Threads D. Stein,D. Shah

Synchronouslygenerated signals are simply changes. Instead the threads library provides a delivered by the kernel to the AcrrvE thread that separatedynamically linked th¡eaddebugging library causedthem. that the debuggerlinks with via dlopen ( ). The Sendínga directed sígnal thread debugging library contains interfaces that allow the A threadcan senda signal to anotherthread in debuggeraccess to generalthread informa- the sameprocess using thr_kill ( The basic tion, without the debugger itself containing ). knowledge meansof sendinga signal to a threadis to sendit to of the threadslibrary implementation.AÍ the the LWP it runs on. If the target thread is not threadslibræy specificknowledge is containedin the thread debugginglibræy. The interfacesin +crrvEl thr_kÍll( ) just postsrhe signal on the the threadin a pendingsignals mask. Wheñ the target thread debugginglibrary allow the debuggerto for thread resumesexecution, it receives the pending example, enumerate the list of threads in the signalsl0. If the target thread is icïve, debuggedprocess, or get/setthe stateand/or registers thr_kilI( of eachthread (nc:nve or not). ) sendsthe signalto the targetthread's 'We LWP via lwp_kill( ). If the targerLWp is block- have a versionof DBX with threadexten- ing the signal (which implies that the thread is sions that dynamically links with this library. The llocking it too), the signal stays pending on the new features allow the programmer to debug a LWP, until the threadunblocks it. TVhilethe signal multi-threadprocess. It will list all threadsin the is pending on the LWP, the thread is temporarily processand their states,whether they are sleeping, bound to the LWP until the signalsa¡e deliveredtô running, active, stopped or zombied. DBX- can the thread. changeits focus to any thread so that its internal SignalSafe Critical Sections state can be analyzedwith the standardDBX com- mands. As statedpreviously, to preventdeadlock in the presenceof signals,critical sectionsthat are reen- References tered in a signal handler in both multi-threaded ,,C applicationsand the threadslibrary should be safe [Cooper1990] E. C. Cooper, R. P. Draves, with_respect to signals. All asynchronoussignals Threads", Departmentof Computer Science, shouldbe maskedduring suchcritical sections. CarnegieMellon University,September 1990. 79921J.R. Eykholt, The threads library signals implementation [Eykholt s. R. Kleiman,S. Bar- ton, R. Faulkner,A. Shivalingiah,M. allows multi-threadedapplications to make critical Smith,D. Stein, J. Voll, M. Weeks, D. Williams, sectionssafe as efficiently as possible via a low "Beyond Multiprocessing: Multitlueading overhead implementation of the the SUnOSKernel", Ptoc. 7992 usENIx thr_sÍgsetmask( interface.If signalsdo not Summer ) Conference,pp. 11-18. occur, thr_sigsetmask( does not result ) in a ,,The pro- systemcall. In this case,it is as fast as just modify, [Faulkner1991] R. Faulkner,R. Gomes, ing the user-levelthread signal mask. cess and ProcessModel in wx SystemV", Proc.1991 UsEMx Winter Confer- The threadslibrary has an even fastermeans of ence. achievingsignal safety for its internal critical sec- 1990]D. Golub, R. Dean, A. Florin, tions.The threadslibrary sets/clearsa special [Golub R. flag in Rashid, "uNIx as an Application program,', the th¡eadsstructure whenever it enter/exits an in-ter- Proc. 1990usENrx Summer nal .Effectively, Conference,pp 82- this flag servesas a 95. signalmask to mask out all signals.If ihe flag is set 1992]Sandeep Khanna, w.hen-a- signal is delivered,the threadslibrary global [Khanna Michael Sebrde, John Zolnowsky, "Realtime Scheduling signal handlerwill defer the signal as describiedin in the abovesection. SunOS5.0", Proc.1992 uspxx WinterConfer- ence. DebuggingThreads POSIX 19921POSIX P1003.4a, "Threads Extension for PortableOperating Systems", IEEE. debuggerthat can control library supported . + 1991]M. L. Powell,S. R. Kleiman,S. Bar- threads requires access to information aUõüt [Powell ttre ton, D. Shah, D. Stein, ,,SunOS threads inside the debuggedprocess. M. Weeks, The normal Multi-threadA¡chitecture", kern-eJ-supporteddebugging interfaces (/proc) Proc. 1991 useNx are Winter Conference. insufficient. One could build completeknowledge of 1990]Lui Sha, Ragunathan p. the threadsimplementation into the debugger,but [Sha Rajkumar,John Iæhoczky, protocols: that would force a re-releaseof the debuggli when- "Priority Inheritance An Approach to Realtime ever some internal threads library data shucture Synchronization',,Vol 39, No 9, September1990, IEeE Transactions on Computers. @ switchcode ensures this by sendingthe pendingsignals to the LWp thatresumes thê thread. \ '92 Summer USENIX - June 8.June 12,lgg2 - San Antonio, TX D. Stein,D. Shah Implementing Lightwetght Threads

IUSO 1990]UNIX SoftwareOperation, "system V Application Binary Interface,SPARC Processor Supplement",UNIX Press. Author Information Dan Stein is currently a memberof technical staff at SunSoft, Inc. where he is one of the developersof the SunOS mult-threadA¡chitecture. He graduatedfrom the University of Wisconsinin 1981with a B.S. in ComputerScience. DevangShah is cunently a Memberof Techni- cal Staff at SunSoft,Inc. He receivedan M.A. in ComputerScience ftom the University of Texas at Austin in 1989 and a B. Tech. in ElectronicsEngg. from the Instituteof Technology,B.H.U., India in 1985.At UT-Austinhe extendedSunOS 3.2 to pro- vide lighnveight processes.Prior to UT-Austin, he workedat Tata ConsultancyServices, Bombay.

Summer'92 USENIX- June8-June L2,1992 - SanAntonio, TX