runtime: make channels work in interrupts

Этот коммит содержится в:
Jaden Weiss 2020-05-29 23:46:58 -04:00 коммит произвёл Ron Evans
родитель aa3481e06a
коммит a4f3457747
13 изменённых файлов: 232 добавлений и 33 удалений

Просмотреть файл

@ -5,6 +5,8 @@ import (
"machine" "machine"
) )
var timerCh = make(chan struct{}, 1)
func main() { func main() {
machine.LED.Configure(machine.PinConfig{Mode: machine.PinOutput}) machine.LED.Configure(machine.PinConfig{Mode: machine.PinOutput})
@ -12,17 +14,18 @@ func main() {
arm.SetupSystemTimer(machine.CPUFrequency() / 10) arm.SetupSystemTimer(machine.CPUFrequency() / 10)
for { for {
machine.LED.Low()
<-timerCh
machine.LED.High()
<-timerCh
} }
} }
var led_state bool
//export SysTick_Handler //export SysTick_Handler
func timer_isr() { func timer_isr() {
if led_state { select {
machine.LED.Low() case timerCh <- struct{}{}:
} else { default:
machine.LED.High() // The consumer is running behind.
} }
led_state = !led_state
} }

Просмотреть файл

@ -1,5 +1,7 @@
package task package task
import "runtime/interrupt"
const asserts = false const asserts = false
// Queue is a FIFO container of tasks. // Queue is a FIFO container of tasks.
@ -10,7 +12,9 @@ type Queue struct {
// Push a task onto the queue. // Push a task onto the queue.
func (q *Queue) Push(t *Task) { func (q *Queue) Push(t *Task) {
i := interrupt.Disable()
if asserts && t.Next != nil { if asserts && t.Next != nil {
interrupt.Restore(i)
panic("runtime: pushing a task to a queue with a non-nil Next pointer") panic("runtime: pushing a task to a queue with a non-nil Next pointer")
} }
if q.tail != nil { if q.tail != nil {
@ -21,12 +25,15 @@ func (q *Queue) Push(t *Task) {
if q.head == nil { if q.head == nil {
q.head = t q.head = t
} }
interrupt.Restore(i)
} }
// Pop a task off of the queue. // Pop a task off of the queue.
func (q *Queue) Pop() *Task { func (q *Queue) Pop() *Task {
i := interrupt.Disable()
t := q.head t := q.head
if t == nil { if t == nil {
interrupt.Restore(i)
return nil return nil
} }
q.head = t.Next q.head = t.Next
@ -34,11 +41,13 @@ func (q *Queue) Pop() *Task {
q.tail = nil q.tail = nil
} }
t.Next = nil t.Next = nil
interrupt.Restore(i)
return t return t
} }
// Append pops the contents of another queue and pushes them onto the end of this queue. // Append pops the contents of another queue and pushes them onto the end of this queue.
func (q *Queue) Append(other *Queue) { func (q *Queue) Append(other *Queue) {
i := interrupt.Disable()
if q.head == nil { if q.head == nil {
q.head = other.head q.head = other.head
} else { } else {
@ -46,6 +55,15 @@ func (q *Queue) Append(other *Queue) {
} }
q.tail = other.tail q.tail = other.tail
other.head, other.tail = nil, nil other.head, other.tail = nil, nil
interrupt.Restore(i)
}
// Empty checks if the queue is empty.
func (q *Queue) Empty() bool {
i := interrupt.Disable()
empty := q.head == nil
interrupt.Restore(i)
return empty
} }
// Stack is a LIFO container of tasks. // Stack is a LIFO container of tasks.
@ -57,19 +75,24 @@ type Stack struct {
// Push a task onto the stack. // Push a task onto the stack.
func (s *Stack) Push(t *Task) { func (s *Stack) Push(t *Task) {
i := interrupt.Disable()
if asserts && t.Next != nil { if asserts && t.Next != nil {
interrupt.Restore(i)
panic("runtime: pushing a task to a stack with a non-nil Next pointer") panic("runtime: pushing a task to a stack with a non-nil Next pointer")
} }
s.top, t.Next = t, s.top s.top, t.Next = t, s.top
interrupt.Restore(i)
} }
// Pop a task off of the stack. // Pop a task off of the stack.
func (s *Stack) Pop() *Task { func (s *Stack) Pop() *Task {
i := interrupt.Disable()
t := s.top t := s.top
if t != nil { if t != nil {
s.top = t.Next s.top = t.Next
t.Next = nil t.Next = nil
} }
interrupt.Restore(i)
return t return t
} }
@ -89,10 +112,13 @@ func (t *Task) tail() *Task {
// Queue moves the contents of the stack into a queue. // Queue moves the contents of the stack into a queue.
// Elements can be popped from the queue in the same order that they would be popped from the stack. // Elements can be popped from the queue in the same order that they would be popped from the stack.
func (s *Stack) Queue() Queue { func (s *Stack) Queue() Queue {
i := interrupt.Disable()
head := s.top head := s.top
s.top = nil s.top = nil
return Queue{ q := Queue{
head: head, head: head,
tail: head.tail(), tail: head.tail(),
} }
interrupt.Restore(i)
return q
} }

Просмотреть файл

@ -102,3 +102,7 @@ func procPin() {
func procUnpin() { func procUnpin() {
arm.EnableInterrupts(procPinnedMask) arm.EnableInterrupts(procPinnedMask)
} }
func waitForEvents() {
arm.Asm("wfe")
}

Просмотреть файл

@ -90,3 +90,11 @@ func procPin() {
func procUnpin() { func procUnpin() {
riscv.EnableInterrupts(procPinnedMask) riscv.EnableInterrupts(procPinnedMask)
} }
func waitForEvents() {
mask := riscv.DisableInterrupts()
if !runqueue.Empty() {
riscv.Asm("wfi")
}
riscv.EnableInterrupts(mask)
}

Просмотреть файл

@ -25,6 +25,7 @@ package runtime
import ( import (
"internal/task" "internal/task"
"runtime/interrupt"
"unsafe" "unsafe"
) )
@ -308,13 +309,17 @@ func (ch *channel) trySend(value unsafe.Pointer) bool {
return false return false
} }
i := interrupt.Disable()
switch ch.state { switch ch.state {
case chanStateEmpty, chanStateBuf: case chanStateEmpty, chanStateBuf:
// try to dump the value directly into the buffer // try to dump the value directly into the buffer
if ch.push(value) { if ch.push(value) {
ch.state = chanStateBuf ch.state = chanStateBuf
interrupt.Restore(i)
return true return true
} }
interrupt.Restore(i)
return false return false
case chanStateRecv: case chanStateRecv:
// unblock reciever // unblock reciever
@ -328,16 +333,21 @@ func (ch *channel) trySend(value unsafe.Pointer) bool {
ch.state = chanStateEmpty ch.state = chanStateEmpty
} }
interrupt.Restore(i)
return true return true
case chanStateSend: case chanStateSend:
// something else is already waiting to send // something else is already waiting to send
interrupt.Restore(i)
return false return false
case chanStateClosed: case chanStateClosed:
interrupt.Restore(i)
runtimePanic("send on closed channel") runtimePanic("send on closed channel")
default: default:
interrupt.Restore(i)
runtimePanic("invalid channel state") runtimePanic("invalid channel state")
} }
interrupt.Restore(i)
return false return false
} }
@ -351,6 +361,8 @@ func (ch *channel) tryRecv(value unsafe.Pointer) (bool, bool) {
return false, false return false, false
} }
i := interrupt.Disable()
switch ch.state { switch ch.state {
case chanStateBuf, chanStateSend: case chanStateBuf, chanStateSend:
// try to pop the value directly from the buffer // try to pop the value directly from the buffer
@ -373,6 +385,7 @@ func (ch *channel) tryRecv(value unsafe.Pointer) (bool, bool) {
ch.state = chanStateEmpty ch.state = chanStateEmpty
} }
interrupt.Restore(i)
return true, true return true, true
} else if ch.blocked != nil { } else if ch.blocked != nil {
// unblock next sender if applicable // unblock next sender if applicable
@ -386,19 +399,24 @@ func (ch *channel) tryRecv(value unsafe.Pointer) (bool, bool) {
ch.state = chanStateEmpty ch.state = chanStateEmpty
} }
interrupt.Restore(i)
return true, true return true, true
} }
interrupt.Restore(i)
return false, false return false, false
case chanStateRecv, chanStateEmpty: case chanStateRecv, chanStateEmpty:
// something else is already waiting to recieve // something else is already waiting to recieve
interrupt.Restore(i)
return false, false return false, false
case chanStateClosed: case chanStateClosed:
if ch.pop(value) { if ch.pop(value) {
interrupt.Restore(i)
return true, true return true, true
} }
// channel closed - nothing to recieve // channel closed - nothing to recieve
memzero(value, ch.elementSize) memzero(value, ch.elementSize)
interrupt.Restore(i)
return true, false return true, false
default: default:
runtimePanic("invalid channel state") runtimePanic("invalid channel state")
@ -447,14 +465,18 @@ type chanSelectState struct {
// This operation will block unless a value is immediately available. // This operation will block unless a value is immediately available.
// May panic if the channel is closed. // May panic if the channel is closed.
func chanSend(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList) { func chanSend(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList) {
i := interrupt.Disable()
if ch.trySend(value) { if ch.trySend(value) {
// value immediately sent // value immediately sent
chanDebug(ch) chanDebug(ch)
interrupt.Restore(i)
return return
} }
if ch == nil { if ch == nil {
// A nil channel blocks forever. Do not schedule this goroutine again. // A nil channel blocks forever. Do not schedule this goroutine again.
interrupt.Restore(i)
deadlock() deadlock()
} }
@ -468,6 +490,7 @@ func chanSend(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList
} }
ch.blocked = blockedlist ch.blocked = blockedlist
chanDebug(ch) chanDebug(ch)
interrupt.Restore(i)
task.Pause() task.Pause()
sender.Ptr = nil sender.Ptr = nil
} }
@ -477,14 +500,18 @@ func chanSend(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList
// The recieved value is copied into the value pointer. // The recieved value is copied into the value pointer.
// Returns the comma-ok value. // Returns the comma-ok value.
func chanRecv(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList) bool { func chanRecv(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList) bool {
i := interrupt.Disable()
if rx, ok := ch.tryRecv(value); rx { if rx, ok := ch.tryRecv(value); rx {
// value immediately available // value immediately available
chanDebug(ch) chanDebug(ch)
interrupt.Restore(i)
return ok return ok
} }
if ch == nil { if ch == nil {
// A nil channel blocks forever. Do not schedule this goroutine again. // A nil channel blocks forever. Do not schedule this goroutine again.
interrupt.Restore(i)
deadlock() deadlock()
} }
@ -498,6 +525,7 @@ func chanRecv(ch *channel, value unsafe.Pointer, blockedlist *channelBlockedList
} }
ch.blocked = blockedlist ch.blocked = blockedlist
chanDebug(ch) chanDebug(ch)
interrupt.Restore(i)
task.Pause() task.Pause()
ok := receiver.Data == 1 ok := receiver.Data == 1
receiver.Ptr, receiver.Data = nil, 0 receiver.Ptr, receiver.Data = nil, 0
@ -511,15 +539,18 @@ func chanClose(ch *channel) {
// Not allowed by the language spec. // Not allowed by the language spec.
runtimePanic("close of nil channel") runtimePanic("close of nil channel")
} }
i := interrupt.Disable()
switch ch.state { switch ch.state {
case chanStateClosed: case chanStateClosed:
// Not allowed by the language spec. // Not allowed by the language spec.
interrupt.Restore(i)
runtimePanic("close of closed channel") runtimePanic("close of closed channel")
case chanStateSend: case chanStateSend:
// This panic should ideally on the sending side, not in this goroutine. // This panic should ideally on the sending side, not in this goroutine.
// But when a goroutine tries to send while the channel is being closed, // But when a goroutine tries to send while the channel is being closed,
// that is clearly invalid: the send should have been completed already // that is clearly invalid: the send should have been completed already
// before the close. // before the close.
interrupt.Restore(i)
runtimePanic("close channel during send") runtimePanic("close channel during send")
case chanStateRecv: case chanStateRecv:
// unblock all receivers with the zero value // unblock all receivers with the zero value
@ -531,6 +562,7 @@ func chanClose(ch *channel) {
// Easy case. No available sender or receiver. // Easy case. No available sender or receiver.
} }
ch.state = chanStateClosed ch.state = chanStateClosed
interrupt.Restore(i)
chanDebug(ch) chanDebug(ch)
} }
@ -541,8 +573,11 @@ func chanClose(ch *channel) {
// TODO: do this in a round-robin fashion (as specified in the Go spec) instead // TODO: do this in a round-robin fashion (as specified in the Go spec) instead
// of picking the first one that can proceed. // of picking the first one that can proceed.
func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelBlockedList) (uintptr, bool) { func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelBlockedList) (uintptr, bool) {
istate := interrupt.Disable()
if selected, ok := tryChanSelect(recvbuf, states); selected != ^uintptr(0) { if selected, ok := tryChanSelect(recvbuf, states); selected != ^uintptr(0) {
// one channel was immediately ready // one channel was immediately ready
interrupt.Restore(istate)
return selected, ok return selected, ok
} }
@ -570,6 +605,7 @@ func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelB
case chanStateRecv: case chanStateRecv:
// already in correct state // already in correct state
default: default:
interrupt.Restore(istate)
runtimePanic("invalid channel state") runtimePanic("invalid channel state")
} }
} else { } else {
@ -582,6 +618,7 @@ func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelB
case chanStateBuf: case chanStateBuf:
// already in correct state // already in correct state
default: default:
interrupt.Restore(istate)
runtimePanic("invalid channel state") runtimePanic("invalid channel state")
} }
} }
@ -594,6 +631,7 @@ func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelB
t.Data = 1 t.Data = 1
// wait for one case to fire // wait for one case to fire
interrupt.Restore(istate)
task.Pause() task.Pause()
// figure out which one fired and return the ok value // figure out which one fired and return the ok value
@ -602,22 +640,27 @@ func chanSelect(recvbuf unsafe.Pointer, states []chanSelectState, ops []channelB
// tryChanSelect is like chanSelect, but it does a non-blocking select operation. // tryChanSelect is like chanSelect, but it does a non-blocking select operation.
func tryChanSelect(recvbuf unsafe.Pointer, states []chanSelectState) (uintptr, bool) { func tryChanSelect(recvbuf unsafe.Pointer, states []chanSelectState) (uintptr, bool) {
istate := interrupt.Disable()
// See whether we can receive from one of the channels. // See whether we can receive from one of the channels.
for i, state := range states { for i, state := range states {
if state.value == nil { if state.value == nil {
// A receive operation. // A receive operation.
if rx, ok := state.ch.tryRecv(recvbuf); rx { if rx, ok := state.ch.tryRecv(recvbuf); rx {
chanDebug(state.ch) chanDebug(state.ch)
interrupt.Restore(istate)
return uintptr(i), ok return uintptr(i), ok
} }
} else { } else {
// A send operation: state.value is not nil. // A send operation: state.value is not nil.
if state.ch.trySend(state.value) { if state.ch.trySend(state.value) {
chanDebug(state.ch) chanDebug(state.ch)
interrupt.Restore(istate)
return uintptr(i), true return uintptr(i), true
} }
} }
} }
interrupt.Restore(istate)
return ^uintptr(0), false return ^uintptr(0), false
} }

Просмотреть файл

@ -29,6 +29,8 @@ package runtime
// Moss. // Moss.
import ( import (
"internal/task"
"runtime/interrupt"
"unsafe" "unsafe"
) )
@ -292,10 +294,36 @@ func GC() {
} }
// Mark phase: mark all reachable objects, recursively. // Mark phase: mark all reachable objects, recursively.
markGlobals()
markStack() markStack()
markGlobals()
// Channel operations in interrupts may move task pointers around while we are marking.
// Therefore we need to scan the runqueue seperately.
var markedTaskQueue task.Queue
runqueueScan:
for !runqueue.Empty() {
// Pop the next task off of the runqueue.
t := runqueue.Pop()
// Mark the task if it has not already been marked.
markRoot(uintptr(unsafe.Pointer(&runqueue)), uintptr(unsafe.Pointer(t)))
// Push the task onto our temporary queue.
markedTaskQueue.Push(t)
}
finishMark() finishMark()
// Restore the runqueue.
i := interrupt.Disable()
if !runqueue.Empty() {
// Something new came in while finishing the mark.
interrupt.Restore(i)
goto runqueueScan
}
runqueue = markedTaskQueue
interrupt.Restore(i)
// Sweep phase: free all non-marked objects and unmark marked objects for // Sweep phase: free all non-marked objects and unmark marked objects for
// the next collection cycle. // the next collection cycle.
sweep() sweep()

Просмотреть файл

@ -2,7 +2,11 @@
package runtime package runtime
import "unsafe" import (
"internal/task"
"runtime/interrupt"
"unsafe"
)
// This garbage collector implementation allows TinyGo to use an external memory allocator. // This garbage collector implementation allows TinyGo to use an external memory allocator.
// It appends a header to the end of every allocation which the garbage collector uses for tracking purposes. // It appends a header to the end of every allocation which the garbage collector uses for tracking purposes.
@ -417,9 +421,9 @@ func (t *memTreap) destroy() {
// This is used to detect if the collector is invoking itself or trying to allocate memory. // This is used to detect if the collector is invoking itself or trying to allocate memory.
var gcrunning bool var gcrunning bool
// activeMem is a treap used to store marked allocations which have already been scanned. // activeMem is a queue used to store marked allocations which have already been scanned.
// This is only used when the garbage collector is running. // This is only used when the garbage collector is running.
var activeMem memTreap var activeMem memScanQueue
func GC() { func GC() {
if gcDebug { if gcDebug {
@ -449,10 +453,27 @@ func GC() {
// These can be quickly compared against to eliminate most false positives. // These can be quickly compared against to eliminate most false positives.
firstPtr, lastPtr = allocations.minAddr(), allocations.maxAddr() firstPtr, lastPtr = allocations.minAddr(), allocations.maxAddr()
// Start by scanning all of the global variables and the stack. // Start by scanning the stack.
markGlobals()
markStack() markStack()
// Scan all globals.
markGlobals()
// Channel operations in interrupts may move task pointers around while we are marking.
// Therefore we need to scan the runqueue seperately.
var markedTaskQueue task.Queue
runqueueScan:
for !runqueue.Empty() {
// Pop the next task off of the runqueue.
t := runqueue.Pop()
// Mark the task if it has not already been marked.
markRoot(uintptr(unsafe.Pointer(&runqueue)), uintptr(unsafe.Pointer(t)))
// Push the task onto our temporary queue.
markedTaskQueue.Push(t)
}
// Scan all referenced allocations, building a new treap with marked allocations. // Scan all referenced allocations, building a new treap with marked allocations.
// The marking process deletes the allocations from the old allocations treap, so they are only queued once. // The marking process deletes the allocations from the old allocations treap, so they are only queued once.
for !scanQueue.empty() { for !scanQueue.empty() {
@ -462,19 +483,29 @@ func GC() {
// Scan and mark all nodes that this references. // Scan and mark all nodes that this references.
n.scan() n.scan()
// Insert this node into the new treap. // Insert this node into the active memory queue.
activeMem.insert(n) activeMem.push(n)
} }
i := interrupt.Disable()
if !runqueue.Empty() {
// Something new came in while finishing the mark.
interrupt.Restore(i)
goto runqueueScan
}
runqueue = markedTaskQueue
interrupt.Restore(i)
// The allocations treap now only contains unreferenced nodes. Destroy them all. // The allocations treap now only contains unreferenced nodes. Destroy them all.
allocations.destroy() allocations.destroy()
if gcAsserts && !allocations.empty() { if gcAsserts && !allocations.empty() {
runtimePanic("failed to fully destroy allocations") runtimePanic("failed to fully destroy allocations")
} }
// Replace the allocations treap with the new treap. // Treapify the active memory queue.
allocations = activeMem for !activeMem.empty() {
activeMem = memTreap{} allocations.insert(activeMem.pop())
}
if gcDebug { if gcDebug {
println("GC finished") println("GC finished")

25
src/runtime/interrupt/interrupt_none.go Обычный файл
Просмотреть файл

@ -0,0 +1,25 @@
// +build !baremetal
package interrupt
// State represents the previous global interrupt state.
type State uintptr
// Disable disables all interrupts and returns the previous interrupt state. It
// can be used in a critical section like this:
//
// state := interrupt.Disable()
// // critical section
// interrupt.Restore(state)
//
// Critical sections can be nested. Make sure to call Restore in the same order
// as you called Disable (this happens naturally with the pattern above).
func Disable() (state State) {
return 0
}
// Restore restores interrupts to what they were before. Give the previous state
// returned by Disable as a parameter. If interrupts were disabled before
// calling Disable, this will not re-enable interrupts, allowing for nested
// cricital sections.
func Restore(state State) {}

Просмотреть файл

@ -216,14 +216,14 @@ func initRTC() {
sam.RTC_MODE0.CTRL.SetBits(sam.RTC_MODE0_CTRL_ENABLE) sam.RTC_MODE0.CTRL.SetBits(sam.RTC_MODE0_CTRL_ENABLE)
waitForSync() waitForSync()
intr := interrupt.New(sam.IRQ_RTC, func(intr interrupt.Interrupt) { rtcInterrupt := interrupt.New(sam.IRQ_RTC, func(intr interrupt.Interrupt) {
// disable IRQ for CMP0 compare // disable IRQ for CMP0 compare
sam.RTC_MODE0.INTFLAG.Set(sam.RTC_MODE0_INTENSET_CMP0) sam.RTC_MODE0.INTFLAG.Set(sam.RTC_MODE0_INTENSET_CMP0)
timerWakeup.Set(1) timerWakeup.Set(1)
}) })
intr.SetPriority(0xc0) rtcInterrupt.SetPriority(0xc0)
intr.Enable() rtcInterrupt.Enable()
} }
func waitForSync() { func waitForSync() {
@ -261,7 +261,10 @@ func sleepTicks(d timeUnit) {
for d != 0 { for d != 0 {
ticks() // update timestamp ticks() // update timestamp
ticks := uint32(d) ticks := uint32(d)
timerSleep(ticks) if !timerSleep(ticks) {
// Bail out early to handle a non-time interrupt.
return
}
d -= timeUnit(ticks) d -= timeUnit(ticks)
} }
} }
@ -280,7 +283,9 @@ func ticks() timeUnit {
} }
// ticks are in microseconds // ticks are in microseconds
func timerSleep(ticks uint32) { // Returns true if the timer completed.
// Returns false if another interrupt occured which requires an early return to scheduler.
func timerSleep(ticks uint32) bool {
timerWakeup.Set(0) timerWakeup.Set(0)
if ticks < 7 { if ticks < 7 {
// Due to around 6 clock ticks delay waiting for the register value to // Due to around 6 clock ticks delay waiting for the register value to
@ -302,8 +307,20 @@ func timerSleep(ticks uint32) {
// enable IRQ for CMP0 compare // enable IRQ for CMP0 compare
sam.RTC_MODE0.INTENSET.SetBits(sam.RTC_MODE0_INTENSET_CMP0) sam.RTC_MODE0.INTENSET.SetBits(sam.RTC_MODE0_INTENSET_CMP0)
for timerWakeup.Get() == 0 { wait:
arm.Asm("wfi") arm.Asm("wfe")
if timerWakeup.Get() != 0 {
return true
}
if hasScheduler {
// The interurpt may have awoken a goroutine, so bail out early.
// Disable IRQ for CMP0 compare.
sam.RTC_MODE0.INTENCLR.SetBits(sam.RTC_MODE0_INTENSET_CMP0)
return false
} else {
// This is running without a scheduler.
// The application expects this to sleep the whole time.
goto wait
} }
} }

Просмотреть файл

@ -20,6 +20,8 @@ import (
const schedulerDebug = false const schedulerDebug = false
var schedulerDone bool
// Queues used by the scheduler. // Queues used by the scheduler.
var ( var (
runqueue task.Queue runqueue task.Queue
@ -114,7 +116,7 @@ func addSleepTask(t *task.Task, duration timeUnit) {
func scheduler() { func scheduler() {
// Main scheduler loop. // Main scheduler loop.
var now timeUnit var now timeUnit
for { for !schedulerDone {
scheduleLog("") scheduleLog("")
scheduleLog(" schedule") scheduleLog(" schedule")
if sleepQueue != nil { if sleepQueue != nil {
@ -135,12 +137,11 @@ func scheduler() {
t := runqueue.Pop() t := runqueue.Pop()
if t == nil { if t == nil {
if sleepQueue == nil { if sleepQueue == nil {
// No more tasks to execute. if asyncScheduler {
// It would be nice if we could detect deadlocks here, because return
// there might still be functions waiting on each other in a }
// deadlock. waitForEvents()
scheduleLog(" no tasks left!") continue
return
} }
timeLeft := timeUnit(sleepQueue.Data) - (now - sleepQueueBaseTime) timeLeft := timeUnit(sleepQueue.Data) - (now - sleepQueueBaseTime)
if schedulerDebug { if schedulerDebug {

Просмотреть файл

@ -19,6 +19,9 @@ func run() {
initAll() initAll()
postinit() postinit()
callMain() callMain()
schedulerDone = true
}() }()
scheduler() scheduler()
} }
const hasScheduler = true

Просмотреть файл

@ -21,3 +21,5 @@ func run() {
postinit() postinit()
callMain() callMain()
} }
const hasScheduler = false

8
src/runtime/wait_other.go Обычный файл
Просмотреть файл

@ -0,0 +1,8 @@
// +build !tinygo.riscv
// +build !cortexm
package runtime
func waitForEvents() {
runtimePanic("deadlocked: no event source")
}