mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
Slot mgr fixes (#613)
*) during shutdown, errors should be 503 *) new inactivity time out for hot queue, we previously kept hot queues in memory forever. *) each hot queue now has a hot launcher to monitor and launch hot containers *) consumers now create a consumer channel with startDequeuer() that can be cancelled via context *) consumers now ping (signal) hot launcher every 200 msecs until they get a slot *) tests for slot queue & mgr
This commit is contained in:
@@ -2,7 +2,6 @@ package agent
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"io"
|
||||
"net/http"
|
||||
"sync"
|
||||
@@ -188,7 +187,7 @@ func (a *agent) Submit(callI Call) error {
|
||||
|
||||
select {
|
||||
case <-a.shutdown:
|
||||
return errors.New("agent shut down")
|
||||
return models.ErrCallTimeoutServerBusy
|
||||
default:
|
||||
}
|
||||
|
||||
@@ -266,74 +265,103 @@ func (a *agent) getSlot(ctx context.Context, call *call) (Slot, error) {
|
||||
|
||||
isHot := protocol.IsStreamable(protocol.Protocol(call.Format))
|
||||
if isHot {
|
||||
// For hot requests, we use a long lived slot queue, which we use to manage hot containers
|
||||
call.slots = a.slotMgr.getHotSlotQueue(call)
|
||||
start := time.Now()
|
||||
|
||||
call.slots.enterState(SlotQueueWaiter)
|
||||
s, err := a.launchHot(ctx, call)
|
||||
call.slots.exitStateWithLatency(SlotQueueWaiter, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
// For hot requests, we use a long lived slot queue, which we use to manage hot containers
|
||||
var isNew bool
|
||||
call.slots, isNew = a.slotMgr.getSlotQueue(call)
|
||||
if isNew {
|
||||
go a.hotLauncher(ctx, call)
|
||||
}
|
||||
|
||||
s, err := a.waitHot(ctx, call)
|
||||
call.slots.exitStateWithLatency(SlotQueueWaiter, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
return s, err
|
||||
}
|
||||
|
||||
return a.launchCold(ctx, call)
|
||||
}
|
||||
|
||||
// launchHot checks with slot queue to see if a new container needs to be launched and waits
|
||||
// for available slots in the queue for hot request execution.
|
||||
func (a *agent) launchHot(ctx context.Context, call *call) (Slot, error) {
|
||||
// hotLauncher is spawned in a go routine for each slot queue to monitor stats and launch hot
|
||||
// containers if needed. Upon shutdown or activity timeout, hotLauncher exits and during exit,
|
||||
// it destroys the slot queue.
|
||||
func (a *agent) hotLauncher(ctx context.Context, callObj *call) {
|
||||
|
||||
isAsync := call.Type == models.TypeAsync
|
||||
// Let use 60 minutes or 2 * IdleTimeout as hot queue idle timeout, pick
|
||||
// whichever is longer. If in this time, there's no activity, then
|
||||
// we destroy the hot queue.
|
||||
timeout := time.Duration(60) * time.Minute
|
||||
idleTimeout := time.Duration(callObj.IdleTimeout) * time.Second * 2
|
||||
if timeout < idleTimeout {
|
||||
timeout = idleTimeout
|
||||
}
|
||||
|
||||
logger := common.Logger(ctx)
|
||||
logger.WithField("launcher_timeout", timeout).Info("Hot function launcher starting")
|
||||
isAsync := callObj.Type == models.TypeAsync
|
||||
|
||||
launchLoop:
|
||||
for {
|
||||
// Check/evaluate if we need to launch a new hot container
|
||||
doLaunch, stats := call.slots.isNewContainerNeeded()
|
||||
common.Logger(ctx).WithField("stats", stats).Debug("checking hot container launch ", doLaunch)
|
||||
|
||||
if doLaunch {
|
||||
ctxToken, tokenCancel := context.WithCancel(context.Background())
|
||||
|
||||
// wait on token/slot/timeout whichever comes first
|
||||
select {
|
||||
case tok, isOpen := <-a.resources.GetResourceToken(ctxToken, call.Memory, isAsync):
|
||||
tokenCancel()
|
||||
if !isOpen {
|
||||
return nil, models.ErrCallTimeoutServerBusy
|
||||
}
|
||||
|
||||
a.wg.Add(1)
|
||||
go a.runHot(ctx, call, tok)
|
||||
case s, ok := <-call.slots.getDequeueChan():
|
||||
tokenCancel()
|
||||
if !ok {
|
||||
return nil, errors.New("slot shut down while waiting for hot slot")
|
||||
}
|
||||
if s.acquireSlot() {
|
||||
if s.slot.Error() != nil {
|
||||
s.slot.Close()
|
||||
return nil, s.slot.Error()
|
||||
}
|
||||
return s.slot, nil
|
||||
}
|
||||
|
||||
// we failed to take ownership of the token (eg. container idle timeout)
|
||||
// try launching again
|
||||
continue launchLoop
|
||||
case <-ctx.Done():
|
||||
tokenCancel()
|
||||
return nil, ctx.Err()
|
||||
select {
|
||||
case <-a.shutdown: // server shutdown
|
||||
return
|
||||
case <-time.After(timeout):
|
||||
if a.slotMgr.deleteSlotQueue(callObj.slots) {
|
||||
logger.Info("Hot function launcher timed out")
|
||||
return
|
||||
}
|
||||
case <-callObj.slots.signaller:
|
||||
}
|
||||
|
||||
// After launching (if it was necessary) a container, now wait for slot/timeout
|
||||
// or periodically reevaluate the launchHot() logic from beginning.
|
||||
isNeeded, stats := callObj.slots.isNewContainerNeeded()
|
||||
logger.WithField("stats", stats).Debug("Hot function launcher stats")
|
||||
if !isNeeded {
|
||||
continue
|
||||
}
|
||||
|
||||
resourceCtx, cancel := context.WithCancel(context.Background())
|
||||
logger.WithField("stats", stats).Info("Hot function launcher starting hot container")
|
||||
|
||||
select {
|
||||
case s, ok := <-call.slots.getDequeueChan():
|
||||
if !ok {
|
||||
return nil, errors.New("slot shut down while waiting for hot slot")
|
||||
case tok, isOpen := <-a.resources.GetResourceToken(resourceCtx, callObj.Memory, isAsync):
|
||||
cancel()
|
||||
if isOpen {
|
||||
a.wg.Add(1)
|
||||
go func(ctx context.Context, call *call, tok ResourceToken) {
|
||||
a.runHot(ctx, call, tok)
|
||||
a.wg.Done()
|
||||
}(ctx, callObj, tok)
|
||||
} else {
|
||||
// this means the resource was impossible to reserve (eg. memory size we can never satisfy)
|
||||
callObj.slots.queueSlot(&hotSlot{done: make(chan struct{}), err: models.ErrCallTimeoutServerBusy})
|
||||
}
|
||||
case <-time.After(timeout):
|
||||
cancel()
|
||||
if a.slotMgr.deleteSlotQueue(callObj.slots) {
|
||||
logger.Info("Hot function launcher timed out")
|
||||
return
|
||||
}
|
||||
case <-a.shutdown: // server shutdown
|
||||
cancel()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// waitHot pings and waits for a hot container from the slot queue
|
||||
func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
||||
|
||||
ch, cancel := call.slots.startDequeuer(ctx)
|
||||
defer cancel()
|
||||
|
||||
for {
|
||||
// send a notification to launcHot()
|
||||
select {
|
||||
case call.slots.signaller <- true:
|
||||
default:
|
||||
}
|
||||
|
||||
select {
|
||||
case s := <-ch:
|
||||
if s.acquireSlot() {
|
||||
if s.slot.Error() != nil {
|
||||
s.slot.Close()
|
||||
@@ -341,13 +369,13 @@ launchLoop:
|
||||
}
|
||||
return s.slot, nil
|
||||
}
|
||||
|
||||
// we failed to take ownership of the token (eg. container idle timeout)
|
||||
// try launching again
|
||||
// we failed to take ownership of the token (eg. container idle timeout) => try again
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
case <-time.After(time.Duration(200) * time.Millisecond):
|
||||
// reevaluate
|
||||
// ping dequeuer again
|
||||
case <-a.shutdown: // server shutdown
|
||||
return nil, models.ErrCallTimeoutServerBusy
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -506,7 +534,6 @@ func (a *agent) prepCold(ctx context.Context, call *call, tok ResourceToken, ch
|
||||
|
||||
func (a *agent) runHot(ctxArg context.Context, call *call, tok ResourceToken) {
|
||||
// We must be careful to only use ctxArg for logs/spans
|
||||
defer a.wg.Done()
|
||||
|
||||
// create a span from ctxArg but ignore the new Context
|
||||
// instead we will create a new Context below and explicitly set its span
|
||||
|
||||
Reference in New Issue
Block a user