<> <> <> <> <<1. All processes execute in a single address space, although this will have to change some day.>> <<2. Processes are lightweight. Besides the process stack there is relatively little state information.>> <<3. Processes are dynamically mapped onto processors by the scheduler, so that processes can be considered virtual processors. We think of the scheduler as executing in the cracks between processes.>> <<4. The scheduler attempts to run higher priority processes in preference to lower priority processes. However, there is no guarantee that a higher priority process will preempt a lower priority process.>> <> <<1. Primitive spin locks (PSLs) are the most primitive locks. PSLs are built out of CStore, are only acquired with reschedule off, and must be pinned. A single processor system should never wait for PSLs, obviously. In a multi-processor system PSLs must never be held long, or they become a bottleneck. The most important (and currently only) PSL is the metaLock, which controls access to the global process state information (especially the meta queues).>> <<2. General spin locks (GSLs) use CStore and directed Yield. If CStore cannot acquire the lock, then the process attempting to acquire the lock donates its cycles to the process holding the lock using directed Yield. If the process holding the lock cannot be run, directed Yield acts like undirected Yield. GSLs can be non-pinned.>> <<3. General queues (GQs) use a GSL to serialize ownership of the queue, and a circular queue of processes to hold the processes that are waiting. See Enqueue and Dequeue for details.>> <<4. Monitor locks (MLs) are built from a GQ and an ownership word, which is used to denote the process owning the lock. See MonitorEntry and MonitorExit for details.>> <<5. Condition variables (CVs) do not assume that a monitor is held, although one is always present. CVs also support timeouts. CVs are built from a GQ and a timeout word that holds the initial value of the timeout period.>> <<6. Interrupt condition variables (ICVs) are the mechanism for turning device signals and interrupts into ICV notifications. ICVs do not assume that a monitor is held, since none is ever present. ICVs also support timeouts. ICVs are built from a CV and a request count for interrupt requests so we do not lose them while not waiting for the ICV. ICVs must be pinned, to allow the scheduler to remove requests from them, and to allow the request count to be changed by devices hooked up to physical memory.>> DragonProcesses: PROGRAM = BEGIN <> <> WordPtr: TYPE = LONG POINTER TO Word; Word: TYPE = PACKED ARRAY [0..31] OF BOOL; WordZero: Word _ ALL[FALSE]; RequestKind: TYPE = [0..31]; RequestWordPtr: TYPE = LONG POINTER TO RequestWord; RequestWord: TYPE = PACKED ARRAY RequestKind OF BOOL; NullRequestWord: RequestWord = ALL[FALSE]; StackRegister: TYPE = MACHINE DEPENDENT {min (0), max (127)}; <> <> MetaQueue: TYPE = LONG POINTER TO Process; MetaQueueRep: TYPE = Process; Queue: TYPE = LONG POINTER TO QueueRep; QueueRep: TYPE = RECORD [ busy: Process, -- busy is a GSL that protects the chain chain: Process -- chain points to the tail of a circular process queue ]; MonitorLock: TYPE = LONG POINTER TO MonitorLockRep; MonitorLockRep: TYPE = RECORD [ queue: QueueRep, -- the GQ of waiting processes owner: Process -- the process owning the lock ]; Condition: TYPE = LONG POINTER TO ConditionRep; ConditionRep: TYPE = RECORD [ queue: QueueRep, -- the GQ of waiting processes timeout: Word -- the timeout initialization ]; InterruptCondition: TYPE = LONG POINTER TO InterruptConditionRep; InterruptConditionRep: TYPE = RECORD [ queue: QueueRep, -- the GQ of waiting processes timeout: Word, -- the timeout initialization requests: Word -- 0 => no requests, # 0 => pending requests ]; <> ProcessPtr: TYPE = LONG POINTER TO Process; Process: TYPE = LONG POINTER TO ProcessRep; ProcessRep: TYPE = RECORD [ queue: Queue, -- pointer to queue that this process is waiting for next: Process, -- next process in above circular queue meta: Process, -- next process in ready queue, timeout queue or page fault queue when: Word, -- when timeout will occur OR page number for fault priority: Priority, -- priority of this process state: ProcessState, -- process state euState: EUstate -- useful registers in EU ]; Priority: TYPE = MACHINE DEPENDENT { slothful (0), -- user-level deep background processing (idle) sluggish (1), -- user-level background processing normal (2), -- user-level normal processing perky (3), -- user-level foreground processing nervous (4), -- system-level ?? processing jumpy (5), -- system-level ?? processing excited (6), -- system-level real-time processing hyper (7) -- system-level emergency processing }; ProcessState: TYPE = MACHINE DEPENDENT { free (0), -- process is on free list running (1), -- process is running (assigned to a processor) ready (2), -- process is ready to run (on ready queue) waitingPage (3), -- process is waiting for page fault (on page fault queue) waitingCV (4), -- process is waiting for CV waitingML (5), -- process is waiting for ML waitingICV (6), -- process is waiting for ICV done (7) -- process is done (waiting for Join) }; EUstateIndex: TYPE = MACHINE DEPENDENT { Carry (0), -- the carry bit (not really a register) Field (1), -- shifter control Hook (2), -- pointer to the youngest frame saved to memory (a nacho) FramesLeft (3) -- frames left before fault occurs }; EUstate: TYPE = ARRAY EUstateIndex OF Word; <> <> <<>> Nacho: TYPE = LONG POINTER TO NachoRep; NachoRep: TYPE = RECORD [ link: Nacho, -- link to the next elder frame in the process stack nextPC: Word, -- the continuation PC for the frame nRegs: Word, -- the # of registers saved in this Nacho others: Nacho, -- the link to the area for more saved registers regs: RegArray -- the saved registers (local variables) ]; RegArray: TYPE = ARRAY Reg OF Word; Reg: TYPE = [0..15]; <> <> <<>> Processor: TYPE = LONG POINTER TO ProcessorRep; ProcessorRep: TYPE = RECORD [ next: Processor, -- next processor in ring orders: ProcessorOrders, -- orders for what to do after reschedule switchTo: Process, -- if orders = switchToGiven, then this is the one to switch to running: Process -- the process currently being run {# NIL} ]; ProcessorOrders: TYPE = MACHINE DEPENDENT { reset (0), -- useful during system init (?) noChange (1), -- ignore the reschedule switchToGiven (2), -- switch to process given by processor.switchTo (NIL => to best) panicStop (3), -- save current process, then spin on these orders stopped (4) -- stopped in response to panicStop }; <> RunawayLimit: Word _ RunawayLimit; <<... arbitrary limit on stack depth to guard against runaway processes. This limit could also be kept on a per-process basis at the cost of extra space. It is left as an unspecified constant.>> <<>> EmergencyLimit: Word _ EmergencyLimit; <<... arbitrary limit on remaining stack frames to ensure that enough is left to save the state information for all running processes. It is left as an unspecified constant.>> <<>> metaLock: Process _ NIL; <> <<>> schedulerLock: Process _ NIL; <> readyQueueArray: ARRAY Priority OF Process; <> pageFaultMetaQueue: MetaQueueRep; <> pageFaultHandlerQueueRep: InterruptConditionRep; pageFaultHandlerQueue: InterruptCondition = @pageFaultHandlerQueueRep; <> timeoutMetaQueue: MetaQueueRep; <> <<>> reqWord: RequestWord _ NullRequestWord; reqPtr: RequestWordPtr = @reqWord; <> requestQueueArray: ARRAY RequestKind OF InterruptConditionRep; <> tailProcessor: Processor _ NIL; <> tailFreeProcess: Process _ NIL; <> panicStopLock: Process _ NIL; <> lowestRunningPriority: Priority _ slothful; <> deferredWakeups: BOOL _ FALSE; <> spinTries: INT _ 8; <<# of times to spin-test a spin-lock. This number should be 0 for a single-processor system, since there is no advantage to extra spins.>> <> RescheduleTrap: PROC = { <<... is called whenever the reschedule interrupt is enabled and either the reschedule line is asserted OR reschedule traps become enabled and there is a deferred reschedule pending. In either case, we start up with critical traps inhibited, and a special return (hand-coded in the RescheduleTrap vector) enables critical traps on exit.>> entryPriority: Priority _ CurrentProcess[].priority; DO <<** Step 1 ** try to grab ownership of the scheduler (if at lowest priority)>> IF entryPriority = lowestRunningPriority AND schedulerLock = NIL THEN <> <<>> IF CStoreProcess[ptr: @schedulerLock, old: NIL, new: CurrentProcess[]] THEN { <> <<** Step 2 ** sample & clear request word>> currentReq: RequestWord; DO currentReq _ reqPtr^; IF currentReq = NullRequestWord THEN EXIT; IF CStoreReq[ptr: reqPtr, old: currentReq, new: NullRequestWord] THEN EXIT; ENDLOOP; <<>> <<** Step 3 ** service requests in currentReq>> AcquireMetaLock[]; -- don't allow changes to the ready queue by other processors IF currentReq # NullRequestWord THEN ServiceRequests[currentReq]; <<** Step 4 ** determine if processor assignments need to be reshuffled>> AssignProcessors[]; <<** Step 5 ** release locks & test for new interrupt requests>> schedulerLock _ NIL; ReleaseMetaLock[]; }; <<** Step 6 ** normal exit, just follow orders >> DO SELECT CurrentProcessor[].orders FROM noChange => EXIT; switchToGiven => { ProcessSwitch[ready, GetL[], CurrentProcessor[].switchTo]; CurrentProcessor[].orders _ noChange; -- reset the orders once followed }; panicStop => { <> SaveProcessState[CurrentProcess[], GetL[]]; CurrentProcessor[].orders _ stopped; -- show that we really are stopped }; ENDCASE; ENDLOOP; <<** Step 7 ** test for residual unserviced interrupts >> IF reqPtr^ = NullRequestWord OR schedulerLock # NIL THEN EXIT; entryPriority _ lowestRunningPriority; <> ENDLOOP; }; <> Fork: PROC [frame: Nacho] RETURNS [new: Process _ NIL] = { <<... forks a new process, which will initially call the procedure at frame.nextPC with arguments given in the frame.>> <> <> <> NotYetImplemented[]; }; Join: PROC [process: Process] RETURNS [frame: Nacho] = { <<... joins an old process, which waits for the process to complete, then returns the return values as a nacho.>> <> <> <> <> NotYetImplemented[]; }; Yield: PROC [nextState: ProcessState _ ready, nextProcess: Process _ NIL, when: Word] = { <<... yields the processor to the nextProcess. If nextProcess = NIL, chooses the "best" process to yield to. The current process will be placed in the nextState. Yield inhibits critical traps, and performs a ProcessSwitch. This routine must be called only when critical traps are enabled.>> InhibitInterrupts[]; CurrentProcess[].when _ when; ProcessSwitch[nextState, GetL[], nextProcess]; <> EnableInterrupts[]; }; Wait: PROC [cond: Condition, lock: MonitorLock _ NIL] = { <<... waits for the condition variable to be the target of a Notify, Broadcast, or Timeout. It is assumed that the given monitor lock is owned, and must be released during the possible pause. The monitor lock must also be re-acquired after the Wait has finished.>> self: Process = CurrentProcess[]; <> Enqueue[@cond.queue, self]; <> IF lock # NIL THEN MonitorExit[lock]; <> Yield[waitingCV, NIL, cond.timeout]; <> IF self.next # NIL THEN DequeueSelf[@cond.queue]; <> IF lock # NIL THEN MonitorEntry[lock]; }; WaitInterrupt: PROC [cond: InterruptCondition] = { <<... waits for the interrupt condition variable to be the target of an interrupt request, or Timeout. Unlike Wait, there are no monitor locks involved (although a version of this XOP could be produced that did release a monitor lock). Also unlike Wait, we have to proceed immediately if interrupts have occurred since the last WaitInterrupt.>> self: Process = CurrentProcess[]; DO <> currentRequests: Word = cond.requests; IF currentRequests # WordZero THEN {cond.requests _ WordZero; RETURN}; <> InhibitInterrupts[]; Enqueue[@cond.queue, self]; EnableInterrupts[]; <> IF cond.requests = WordZero THEN Yield[waitingICV, NIL, cond.timeout]; <> DequeueSelf[@cond.queue]; ENDLOOP; }; Notify: PROC [cond: Condition, lock: MonitorLock _ NIL] = { <<... notifies the condition variable that the condition has occurred.>> best: Process = Dequeue[@cond.queue]; IF best # NIL THEN MakeReady[best]; <> }; Broadcast: PROC [cond: Condition, lock: MonitorLock _ NIL] = { <<... notifies all waiting processes that the condition has occurred. To eliminate race problems, all of the waiting processes are made ready while holding onto the queue.>> tail: Process _ cond.queue.chain; IF tail # NIL THEN { AcquireQueue[@cond.queue]; IF (tail _ cond.queue.chain) # NIL THEN { cond.queue.chain _ NIL; DO <> this: Process = tail.next; next: Process _ this.next; tail.next _ next; this.queue _ NIL; this.next _ NIL; MakeReady[this]; IF this = next THEN EXIT; -- we just removed the last one ENDLOOP; }; ReleaseQueue[@cond.queue]; }; }; MonitorEntry: PROC [lock: MonitorLock] = { <<... is performed at the entry to any monitored procedure to lock up the monitor for exclusive access. This operation is performed frequently, so it should be fast in the case where the monitor is not already owned.>> self: Process = CurrentProcess[]; DO <> IF CStoreProcess[ptr: @lock.owner, old: NIL, new: self] THEN RETURN; <<>> <> Enqueue[@lock.queue, self]; <<>> <> IF lock.owner # NIL THEN Yield[waitingML, NIL, WordZero] ELSE DequeueSelf[@lock.queue]; ENDLOOP; }; MonitorExit: PROC [lock: MonitorLock] = { <<... frees up the monitor lock. If there is a process waiting the monitor, dequeue the process from the monitor queue and place it in the ready queue. The current coding does nothing if we do not own the lock. Should we die instead of doing nothing?>> IF lock.owner = CurrentProcess[] THEN { lock.owner _ NIL; <> IF lock.queue.chain # NIL THEN { <> next: Process = Dequeue[@lock.queue]; IF next # NIL THEN MakeReady[next]; }; IF deferredWakeups THEN RaiseReschedule[]; <> }; }; <> ShuffleReadyQueues: PROC = { <> InhibitInterrupts[]; AcquireMetaLock[]; <> FOR processor: Processor _ tailProcessor.next, processor.next DO SELECT processor.orders FROM noChange => { <> processor.orders _ switchToGiven; processor.switchTo _ NIL; }; ENDCASE; IF processor = tailProcessor THEN EXIT; ENDLOOP; <> FOR priority: Priority DECREASING IN Priority DO <> tail: Process = readyQueueArray[priority]; IF tail # NIL THEN readyQueueArray[priority] _ tail.meta; ENDLOOP; RaiseReschedule[]; ReleaseMetaLock[]; EnableInterrupts[]; }; CausePanicStop: PROC = { <> self: Process = CurrentProcess[]; InhibitInterrupts[]; WHILE NOT CStoreProcess[ptr: @panicStopLock, old: NIL, new: self] DO <> WHILE panicStopLock # NIL DO SaveProcessState[self, GetL[]]; ENDLOOP; ENDLOOP; SaveProcessState[self, GetL[]]; <> DO done: BOOL _ TRUE; AcquireMetaLock[]; FOR processor: Processor _ tailProcessor.next, processor.next DO IF processor # CurrentProcessor[] THEN SELECT processor.orders FROM noChange => { <> processor.orders _ panicStop; processor.switchTo _ NIL; }; stopped => done _ FALSE; ENDCASE => done _ FALSE; IF processor = tailProcessor THEN EXIT; ENDLOOP; IF done THEN EXIT; ReleaseMetaLock[]; RaiseReschedule[]; ENDLOOP; <> }; ReleasePanicStop: PROC = { <> FOR processor: Processor _ tailProcessor.next, processor.next DO SELECT processor.orders FROM stopped => processor.orders _ noChange; ENDCASE; IF processor = tailProcessor THEN EXIT; ENDLOOP; <> panicStopLock _ NIL; ReleaseMetaLock[]; EnableInterrupts[]; }; <> ProcessSwitch: PROC [nextState: ProcessState, limitL: StackRegister, nextProcess: Process _ NIL] = INLINE { <<... saves the state of the current process. If the process is not waiting, then it is placed on the ready queue. If nextProcess # NIL then it is the next process to run. Otherwise, the "best" ready process is chosen. During this sequence, we assume that critical traps are disabled. Note that if nextProcess # NIL, we must switch to it if it is ready, even if the current process is "better" (see comments in AcquireQueue to see why we require this).>> <<>> self: Process = CurrentProcess[]; SaveProcessState[self, limitL]; AcquireMetaLock[]; -- don't allow changes to the ready queue by other processors SELECT nextState FROM running => GO TO bailOut; ready => EnqueueReady[self]; waitingPage => EnqueuePageFault[self]; waitingML => { <> IF self.next = NIL THEN GO TO bailOut; self.state _ nextState}; waitingCV, waitingICV => { <> IF self.next = NIL THEN GO TO bailOut; IF Positive[self.when] THEN EnqueueMeta[@timeoutMetaQueue, self]; self.state _ nextState}; ENDCASE; SELECT TRUE FROM nextProcess = NIL OR nextProcess.state # ready => { <> nextProcess _ PickBestProcess[]; IF nextProcess = self THEN GO TO bailOut}; nextProcess.state = waitingPage => { <> self.when _ nextProcess.when; EnqueuePageFault[self]; nextProcess _ PickBestProcess[]}; ENDCASE; <> <> <<>> <> DequeueMeta[@readyQueueArray[nextProcess.priority], nextProcess]; nextProcess.state _ running; CurrentProcessor[].running _ nextProcess; SELECT self.priority FROM nextProcess.priority => {}; lowestRunningPriority => { <> new: Priority _ tailProcessor.running.priority; FOR each: Processor _ tailProcessor.next, each.next WHILE each # tailProcessor DO IF new > each.running.priority THEN new _ each.running.priority; ENDLOOP; lowestRunningPriority _ new; }; ENDCASE; <> ReleaseMetaLock[]; RestoreProcess[nextProcess]; EXITS bailOut => ReleaseMetaLock[]; <> }; MakeReady: PROC [process: Process] = { <<... places the given process on the ready queue using the meta link, provided that the process was waiting for a queue. Also removes process from the timeout queue if it is there. On entry, we require that critical traps must NOT be disabled.>> InhibitInterrupts[]; AcquireMetaLock[]; IF process.queue = NIL THEN SELECT process.state FROM waitingCV, waitingML, waitingICV => { DequeueMeta[@timeoutMetaQueue, process]; <> EnqueueReady[process]; <> IF process.priority > lowestRunningPriority THEN deferredWakeups _ TRUE; <> }; ENDCASE; ReleaseMetaLock[]; EnableInterrupts[]; }; Enqueue: PROC [q: Queue, p: Process] = INLINE { <<... places the process on the queue in FIFO order. Priority processing is the responsibility of Dequeue.>> AcquireQueue[q]; {tail: Process = q.chain; IF tail = NIL THEN p.next _ p ELSE {p.next _ tail.next; tail.next _ p}; q.chain _ p; p.queue _ q}; ReleaseQueue[q]; }; Dequeue: PROC [q: Queue] RETURNS [best: Process _ NIL] = INLINE { <<... performs dequeue by process priority. We perform a linear search of every process in the queue to find the best. The best process in the queue is the first one we find with the maximum priority. Performing the priority testing in Dequeue is more expensive for the case where there are several things on the queue, but it allows us to change process priorities without altering the queue. If the process we find has also timed out, we have to handle the case elsewhere, but it is OK for Dequeue to dequeue such a process. Dequeue does not remove such a process from the timeout queue.>> lag: Process _ q.chain; IF lag = NIL THEN RETURN; -- quick kill for nothing in the queue AcquireQueue[q]; lag _ q.chain; -- have to refetch, in case it changed IF lag # NIL THEN { tail: Process = lag; best _ lag.next; IF lag = best THEN q.chain _ NIL <> ELSE { head: Process _ best; -- loop invariant: lag.next = best this: Process _ head; DO next: Process _ this.next; IF next = head THEN EXIT; -- we have wrapped IF next.priority > best.priority THEN {lag _ this; best _ next}; <> this _ next; ENDLOOP; IF tail = best THEN q.chain _ lag; <> lag.next _ best.next; }; best.queue _ NIL; best.next _ NIL; -- the last change, due to test in ProcessSwitch }; ReleaseQueue[q]; }; DequeueSelf: PROC [q: Queue] = { <<... dequeues the current process from the given queue. This operation has no effect if the process was not in the queue.>> self: Process = CurrentProcess[]; IF self.next # NIL THEN { lag: Process _ q.chain; IF lag # NIL THEN { AcquireQueue[q]; IF (lag _ q.chain) # NIL THEN { tail: Process = lag; DO p: Process = lag.next; IF p = self THEN { IF p = lag THEN q.chain _ NIL <> ELSE { IF p = tail THEN q.chain _ lag; <> lag.next _ p.next; }; self.queue _ NIL; p.next _ NIL; -- the last change, due to test in ProcessSwitch EXIT; }; lag _ p; IF lag = tail THEN EXIT; ENDLOOP; }; ReleaseQueue[q]; }; }; }; AcquireQueue: PROC [q: Queue] = INLINE { <<... grabs ownership of the given queue. Performs Yield[ready, owner] until queue is owned. This directed Yield is necessary to overcome nasty interactions between spin locks and preemptive scheduling. To avoid premature Yield on multi-processor systems, we spin a short time waiting for the spin lock to clear before trying the more expensive Yield.>> WHILE NOT CStoreProcess[ptr: @q.busy, old: NIL, new: CurrentProcess[]] DO owner: Process _ q.busy; THROUGH [0..spinTries) WHILE owner # NIL DO owner _ q.busy; ENDLOOP; IF owner # NIL THEN Yield[ready, owner, WordZero]; <> ENDLOOP; }; ReleaseQueue: PROC [q: Queue] = INLINE { <<... releases ownership of the queue. Assumes that the current process owns the queue.>> q.busy _ NIL; }; <> AcquireMetaLock: PROC = INLINE { <<... grabs ownership of all of the meta queues. On entry, we require that critical traps are disabled, of course. On exit, we hold the meta lock.>> WHILE NOT CStoreProcess[ptr: @metaLock, old: NIL, new: CurrentProcess[]] DO <> WHILE metaLock # NIL DO ENDLOOP; ENDLOOP; }; SaveProcessState: PROC [process: Process, limitL: StackRegister] = INLINE { <<... saves frames and process info from the current processor for the given process. On entry, we require that critical traps are disabled. On exit, there are no frames in the IFU stack (so a return will provoke a stack underflow). Note that this procedure must be an INLINE to keep from putting another frame on the stack.>> <> <> SaveStack[limitL]; <<>> <> FOR which: EUstateIndex IN EUstateIndex DO process.euState[which] _ EUFetchSpecial[which]; ENDLOOP; <> IF NOT Positive[process.euState[FramesLeft]] THEN <> NotYetImplemented[]; }; RestoreProcess: PROC [process: Process] = INLINE { <<... restores the given process to a running state. On entry, we require that critical traps are disabled, and process.state = running. Note that this procedure must be an INLINE to keep from putting another frame on the stack.>> <<>> <> FOR which: EUstateIndex IN EUstateIndex DO EUStoreSpecial[which, process.euState[which]]; ENDLOOP; }; <> ServiceRequests: PROC [req: RequestWord] = INLINE { <<... services requests in the req word by tweaking the appropriate interrupt condition queues. This loop is not the fastest way to do this, but it is not worth improving unless we find out that it is taking a lot of time.>> FOR reqx: RequestKind IN RequestKind DO IF req[reqx] THEN { who: Process; cond: InterruptCondition = @requestQueueArray[reqx]; cond.requests _ ALL[TRUE]; who _ Dequeue[@cond.queue]; IF who # NIL THEN SELECT who.state FROM waitingCV, waitingML, waitingICV => { DequeueMeta[@timeoutMetaQueue, who]; EnqueueReady[who]; }; ENDCASE; req[reqx] _ FALSE; IF req = NullRequestWord THEN EXIT; }; ENDLOOP; }; AssignProcessors: PROC = { <<... sets processor orders based on whether the a ready process is "better" than any running processes; we run through the ready processes in highest to lowest order. At the end of the reassignment, if we have to change the orders of any processor other than our own then we raise reschedule to get the other processors to do their switching.>> needReschedule: BOOL _ FALSE; current: Processor = CurrentProcessor[]; deferredWakeups _ FALSE; <> FOR priority: Priority DECREASING IN Priority WHILE priority > lowestRunningPriority DO tail: Process = readyQueueArray[priority]; IF tail # NIL THEN { FOR process: Process _ tail.next, process.next DO best: Processor _ NIL; FOR processor: Processor _ current, processor.next DO SELECT processor.orders FROM switchToGiven => <> IF processor.switchTo = process THEN EXIT; noChange => <> SELECT TRUE FROM processor.running.priority >= process.priority => {}; <> best = NIL OR processor.running.priority < best.running.priority => <> best _ processor; ENDCASE; ENDCASE; IF processor.next = current THEN { <> IF best = NIL THEN GO TO done; <> <> best.switchTo _ process; best.orders _ switchToGiven; IF best # CurrentProcessor[] THEN needReschedule _ TRUE; EXIT; }; ENDLOOP; IF process = tail THEN EXIT; ENDLOOP; EXITS done => EXIT; }; ENDLOOP; IF needReschedule THEN RaiseReschedule[]; }; PickBestProcess: PROC RETURNS [best: Process _ NIL] = INLINE { <<... returns the best process in the ready queue and returns it. On entry, we require that critical traps are disabled and that the metaLock is held. No change is made to the process selected.>> FOR priority: Priority DECREASING IN Priority DO tail: Process = readyQueueArray[priority]; IF tail # NIL THEN RETURN [tail.next]; ENDLOOP; }; ReleaseMetaLock: PROC = INLINE { <<... releases ownership of the meta lock. On entry, we require that critical traps are disabled and that the metaLock is held. On exit, the metaLock is not held, although critical traps are still disabled.>> metaLock _ NIL; }; EnqueueReady: PROC [process: Process] = INLINE { <<... places the given process on the ready queue using the meta link. On entry, we require that critical traps are disabled and that the metaLock is held. On exit, process.state = ready.>> EnqueueMeta[@readyQueueArray[process.priority], process]; process.state _ ready; }; EnqueuePageFault: PROC [process: Process] = INLINE { <<... places the given process on the page fault queue using the meta link. On entry, we require that critical traps are disabled and that the metaLock is held. On exit, process.state = waitingPage.>> EnqueueMeta[@pageFaultMetaQueue, process]; process.state _ waitingPage; }; EnqueueMeta: PROC [metaQueue: MetaQueue, process: Process] = INLINE { <<... places the given process on the given meta queue using the meta link. On entry, we require that critical traps are disabled and that the metaLock is held.>> IF process.meta = NIL THEN { tail: Process = metaQueue^; IF tail = NIL THEN process.meta _ process ELSE {process.meta _ tail.meta; tail.meta _ process}; metaQueue^ _ process; }; }; DequeueMeta: PROC [metaQueue: MetaQueue, process: Process] = INLINE { <<... removes the given process from the given meta queue (timeout, ready, pageFault). On entry, we require that critical traps must be disabled and that the metaLock is held. The process state is not changed.>> IF process.meta # NIL THEN { <> tail: Process = metaQueue^; IF tail # NIL THEN { lag: Process _ tail; DO p: Process _ lag.meta; IF p = process THEN { IF lag = p THEN metaQueue^ _ NIL <> ELSE { IF p = tail THEN metaQueue^ _ lag; <> lag.meta _ p.meta; }; p.meta _ NIL; EXIT; }; lag _ p; IF lag = tail THEN EXIT; ENDLOOP; }; }; }; <> InhibitInterrupts: PROC [] = { <<... inhibits interrupts>> NotYetImplemented[]; }; EnableInterrupts: PROC [] = { <<... enables interrupts>> NotYetImplemented[]; }; CStoreProcess: PROC [ptr: ProcessPtr, old,new: Process] RETURNS [stored: BOOL] = { <<... atomically performs {IF (stored _ ptr^ = old) THEN ptr^ _ new}.>> IF (stored _ ptr^ = old) THEN ptr^ _ new; }; CStoreWord: PROC [ptr: WordPtr, old,new: Word] RETURNS [stored: BOOL] = { <<... atomically performs {IF (stored _ ptr^ = old) THEN ptr^ _ new}.>> IF (stored _ ptr^ = old) THEN ptr^ _ new; }; CStoreReq: PROC [ptr: RequestWordPtr, old,new: RequestWord] RETURNS [stored: BOOL] = { <<... atomically performs {IF (stored _ ptr^ = old) THEN ptr^ _ new}.>> IF (stored _ ptr^ = old) THEN ptr^ _ new; }; RaiseReschedule: PROC = { <<... raises the Reschedule interrupt, which, if disabled, will occur when it becomes enabled again.>> deferredWakeups _ FALSE; NotYetImplemented[]; }; CurrentProcess: PROC RETURNS [Process _ NIL] = { <<... returns the current process. This is likely to be an Aux register.>> NotYetImplemented[]; }; CurrentProcessor: PROC RETURNS [Processor _ NIL] = { <<... releases ownership of the queue. This is likely to be an Aux register.>> NotYetImplemented[]; }; Positive: PROC [x: Word] RETURNS [BOOL] = { <<... returns x > 0. Currently ignore differences between halfword order on Dragons and Dorados.>> RETURN [LOOPHOLE[x, INT] > 0]; }; VanillaAdd: PROC [x,y: Word] RETURNS [Word _ WordZero] = { <<... returns x+y, no checking for overflow, no change to carry.>> NotYetImplemented[]; }; VanillaSub: PROC [x,y: Word] RETURNS [Word _ WordZero] = { <<... returns x-y, no checking for overflow, no change to carry.>> NotYetImplemented[]; }; EUFetchSpecial: PROC [which: EUstateIndex] RETURNS [Word _ WordZero] = { <<... fetches a word from the given special EU register.>> NotYetImplemented[]; }; EUStoreSpecial: PROC [which: EUstateIndex, x: Word] = { <<... stores a word into the given special EU register.>> NotYetImplemented[]; }; GetL: PROC RETURNS [StackRegister _ min] = { <<... returns the current value of L for the caller. This must be called with traps disabled to avoid having L move around after it has been sampled.>> NotYetImplemented[]; }; SaveStack: PROC [limitL: StackRegister] = INLINE { <<... saves the stack (updates the current value of the hook EU register), returns the number of frames saved. Also updates the FramesLeft EU register. For excruciating details, see DragonStackSave.mesa.>> NotYetImplemented[]; }; NotYetImplemented: PROC = { <<... marks operations with missing implementation details.>> NotYetImplemented[]; }; <> <> Init: PROC = { NotYetImplemented[]; }; END. Differences Between PrincOps and DragOps Note: this section requires knowledge of the PrincOps architecture. I freely admit that my knowledge of said architecture is spotty at best. This section is also incomplete. Multiple processors With current PrincOps machines atomicity is acheived through a combination of locks, inhibition of interrupts, priority levels, and non-interruptibility of non-faulting microcode routines. In the DragOps arhcitecture, only locks (based on memory bus arbitration) and inhibition of interrupts for a single processor (based on IFU inhibition of the Reschedule interrupt) can be used as sources of atomicity. This complicates life, you betcha. Memory vs. time For PrincOps, memory was assumed to be expensive, and cycles were assumed to be expendable if it saved memory. For example, a process block costs only 8 bytes when no state vector is used, and only costs 36 additional bytes when a state vector is used (the unusual case). The DragOps, memory is assumed to be relatively cheap, and cycles should only be spent to avoid gross memory costs. For example, a process block costs at least 40 bytes in the normal case, and there is no exceptional case. The ready queue For PrincOps, the head of the ready queue is the currently running process. For DragOps, running processes are not on the ready queue at all. Instead, running processes are located through the current ring of processors. The DragOps ready queue is special in that it is actually an array of metaqueues, one for each priority level. This bounds the search for the best ready process to a time proportional to the number of priority levels (small), yet also allows us to place processes on the ready queue in unit time. Queues and meta queues For PrincOps, all queues are circular queues of processes linked through a particular word in the PSB (Process State Block). Condition variables, monitor locks, and even the ready queue are all queues where the queue pointer points at the tail of the processor queue. For DragOps, there are Queues and MetaQueues. Although both are circular queues, and both point at the tail, the link words are different. A DragOps process can be queued at the same time on both a queue and a metaqueue. DragOps queues are used by "normal" Mesa monitors and condition variables. MetaQueues are used for more specialized reasons, like the ready queue (waiting for processor), the pageFault queue (waiting for a page to be brought into memory), and the timeout queue (waiting for some time to expire). Queues need not be pinned, while MetaQueues must be pinned. The scheduler is permitted to move processes between MetaQueues, but is not permitted to touch queues. All of this is done in the name of simplicity, you betcha. Monitor locks & condition variables For PrincOps, monitor locks and condition variables are managed by microcode that has carefully been constructed to not take interrupts at awkward times. For DragOps, managing monitor lock and condition queues is too complex to be performed in a single instruction, so each monitor lock and condition has a spin lock that must be acquired before the queue can be examined or modified. A potential problem with this scheme is that a higher-priority process may attempt to acquire the spin lock while a lower-priority process possess the lock. Therefore, each spin lock has the address of the owning process, so a process that fails to acquire the lock can yiled control directly to the owning process. Although this is technically a violation of preemptive scheduling, it is necessary to prevent livelock (where no progress can be made because of the higher-priority process spinning). State vectors For PrincOps, the eval stack and other cleanup information is contained in a state vector. Minimal stack restrictions on monitor entry and condition wait ensure that the eval stack is empty when such processes go to sleep, which tends to keep the number of state vectors low. For DragOps, there are no state vectors. When a process is about to go to sleep, it saves the frames in the IFU and EU stacks to memory, saves the appropriate EU registers to the process block, and puts itself on the appropriate metaqueue (if any). The only problem is ensuring that there are sufficient nachos to accomodate the stack of the process. The current thinking is to have an emergency-level process put every other process to sleep when frames are about to run out. When that has been done, a world-swap debug can take place. An improvement on this scheme can be made if there is an error generated for a particular process when it has used "too many" frames. Since this is typically due to a process executing unbounded recursion, we can tolerate at least a few runaways without having to world-swap. In both cases, we have to have some mechanism for keeping track of how many frames are in use by a given process, and how many are left in the global pool of frames. Details are yet to be specified. Costs For PrincOps, a process switch is on the order of 5 procedure call times. A procedure call (and return) is on the order of 5 microseconds (on the Dorado). For DragOps, a process switch is on the order of 100 procedure call times (very rough estimate). Partly, this is because procedure calls are about 10 times faster, and partly this is because there is more per process state to save and restore. Timeout For PrincOps, a timeout occurs when the clock ticks and some PSB that is awaiting timeout has its wait count go to 0. All PSBs have to be touched at every tick, although this is done via microcode. For DragOps, at each tick every process in timeoutMetaQueue has its when field decremented. If that field reaches 0, the process is placed in the ready queue. It then becomes the responsibility of the process to remove itself from the appropriate condition variable. This makes page faults (if any) due to touching the condition variable occur in the "right" process. Only PSBs waiting for timeout have to be touched at every tick. Things to do This section is a list of things that still need some design work. Fork / join These need to be coded in greater detail. Page faults How will the fault handler work and place things to sleep waiting for page fault? Where do we get the address of the word touched by the faulting process? Soft and hard emergency limits on frames These need support in the scheduler, as well as in stack save/restore. Reaching the soft limit (per process) should raise an error. This is just an attempt to restrain runaway processes. Reaching the hard limit will cause system shutdown and a call to the world-swap debugger. This probably will only happen if a significant number of runaways occur. Timeout mechanism In current Cedar relatively few condition variables have timeouts (the default is not to have timeouts). The current design for the timeout queue is based on the assumption that this situation will continue on Dragon. The current design has O(n) insertion and deletion of entries from the queue, and O(1) testing, where n is the number of processes waiting for timeout. A heapsort-style priority queue has O(n log n) insertion and deletion, and O(1) testing, but it also has more complicated code, and needs an auxilliary pinned data structure (1 word / process). The current simple design will be used until it becomes a bottleneck. Should the timeout "process" just be part of the scheduler? How will elapsed time be determined, and to what accuracy? Just for the record, there are three clocks of current interest to Cedar: 1: Time of day clock. This need not have resolution of better than a second, and it need not be locally stable, but it should have very good long-term accuracy. 2: Timeout clock. This need not have great accuracy, but it must generate interrupts, and it should have resolution of 10-100 milliseconds. More resolution is OK provided that interrupts don't occur more often than every 10 milliseconds. 3: Elapsed time clock. This need not have great long-term accuracy, but it should have very good short-term accuracy, have roughly microsecond resolution, and it must be quite cheap to read. This clock can easily synthesize the Timeout clock. Stack trace SafeStorage needs a way to trace out the process stacks atomically. That is, all possible references in the process stacks must be discovered without any reference containing processes making headway. Dump frames are already pinned, and the panicStop mechanism can be used to stop all processes except for the current process (we may also want to have an stopRC command that would stop all processes that had not declared themselves to be non-RC). We can allocate frame extensions out of a special pool of memory that allows scanning. Possibly we should also make this pool pinned? Reschedule & page fault Setting L at procedure essentially transfers registers from the called frame to the calling frame, as far as stack save and restore are concerned. It will only work properly if both the calling frame and the called frame are in the EU (and IFU) registers. If the various compilers (and assembly language coders) guarantee that no calls are made before setting L, then there are only two sources of having the calling frame migrated into memory - reschedule & page fault. For these traps, if the youngest frame below the trap is P, and the next youngest is Q, and Q has zero registers, then both P and Q have to be restored before returning from the trap.