mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
fn: container initialization monitoring (#1288)
Container initialization phase consumes resource tracker resources (token), during lengthy operations. In order for agent stability/liveness, this phase has to be evictable/cancelable and time bounded. With this change, introducing a new system wide environment setting to bound the time spent in container initialization phase. This phase includes docker-pull, docker-create, docker-attach, docker-start and UDS wait operations. This initialization period is also now considered evictable.
This commit is contained in:
@@ -359,17 +359,30 @@ func (a *agent) getSlot(ctx context.Context, call *call) (Slot, error) {
|
||||
|
||||
call.slots, isNew = a.slotMgr.getSlotQueue(call.slotHashId)
|
||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
||||
if isNew {
|
||||
go a.hotLauncher(ctx, call)
|
||||
|
||||
// setup slot caller with a ctx that gets cancelled once waitHot() is completed.
|
||||
// This allows runHot() to detect if original caller has been serviced by
|
||||
// another container or if original caller was disconnected.
|
||||
caller := &slotCaller{}
|
||||
{
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
|
||||
caller.done = ctx.Done()
|
||||
caller.notify = make(chan error)
|
||||
}
|
||||
s, err := a.waitHot(ctx, call)
|
||||
|
||||
if isNew {
|
||||
go a.hotLauncher(ctx, call, caller)
|
||||
}
|
||||
s, err := a.waitHot(ctx, call, caller)
|
||||
return s, err
|
||||
}
|
||||
|
||||
// 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, call *call) {
|
||||
func (a *agent) hotLauncher(ctx context.Context, call *call, caller *slotCaller) {
|
||||
// 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.
|
||||
@@ -388,12 +401,9 @@ func (a *agent) hotLauncher(ctx context.Context, call *call) {
|
||||
ctx, span := trace.StartSpan(ctx, "agent_hot_launcher")
|
||||
defer span.End()
|
||||
|
||||
var notifyChan chan error
|
||||
|
||||
for {
|
||||
ctx, cancel := context.WithTimeout(ctx, timeout)
|
||||
a.checkLaunch(ctx, call, notifyChan)
|
||||
notifyChan = nil
|
||||
a.checkLaunch(ctx, call, *caller)
|
||||
|
||||
select {
|
||||
case <-a.shutWg.Closer(): // server shutdown
|
||||
@@ -405,7 +415,7 @@ func (a *agent) hotLauncher(ctx context.Context, call *call) {
|
||||
logger.Debug("Hot function launcher timed out")
|
||||
return
|
||||
}
|
||||
case notifyChan = <-call.slots.signaller:
|
||||
case caller = <-call.slots.signaller:
|
||||
cancel()
|
||||
}
|
||||
}
|
||||
@@ -420,7 +430,7 @@ func tryNotify(notifyChan chan error, err error) {
|
||||
}
|
||||
}
|
||||
|
||||
func (a *agent) checkLaunch(ctx context.Context, call *call, notifyChan chan error) {
|
||||
func (a *agent) checkLaunch(ctx context.Context, call *call, caller slotCaller) {
|
||||
curStats := call.slots.getStats()
|
||||
isNB := a.cfg.EnableNBResourceTracker
|
||||
if !isNewContainerNeeded(&curStats) {
|
||||
@@ -471,19 +481,19 @@ func (a *agent) checkLaunch(ctx context.Context, call *call, notifyChan chan err
|
||||
if tok != nil {
|
||||
if tok.Error() != nil {
|
||||
if tok.Error() != CapacityFull {
|
||||
tryNotify(notifyChan, tok.Error())
|
||||
tryNotify(caller.notify, tok.Error())
|
||||
} else {
|
||||
needMem, needCpu := tok.NeededCapacity()
|
||||
notifyChans = a.evictor.PerformEviction(call.slotHashId, needMem, uint64(needCpu))
|
||||
// For Non-blocking mode, if there's nothing to evict, we emit 503.
|
||||
if len(notifyChans) == 0 && isNB {
|
||||
tryNotify(notifyChan, tok.Error())
|
||||
tryNotify(caller.notify, tok.Error())
|
||||
}
|
||||
}
|
||||
} else if a.shutWg.AddSession(1) {
|
||||
go func() {
|
||||
// NOTE: runHot will not inherit the timeout from ctx (ignore timings)
|
||||
a.runHot(ctx, call, tok, state)
|
||||
a.runHot(ctx, caller, call, tok, state)
|
||||
a.shutWg.DoneSession()
|
||||
}()
|
||||
// early return (do not allow container state to switch to ContainerStateDone)
|
||||
@@ -510,7 +520,7 @@ func (a *agent) checkLaunch(ctx context.Context, call *call, notifyChan chan err
|
||||
}
|
||||
|
||||
// waitHot pings and waits for a hot container from the slot queue
|
||||
func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
||||
func (a *agent) waitHot(ctx context.Context, call *call, caller *slotCaller) (Slot, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "agent_wait_hot")
|
||||
defer span.End()
|
||||
|
||||
@@ -519,15 +529,13 @@ func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
||||
|
||||
ch := call.slots.startDequeuer(ctx)
|
||||
|
||||
notifyChan := make(chan error)
|
||||
|
||||
// 1) if we can get a slot immediately, grab it.
|
||||
// 2) if we don't, send a signaller every x msecs until we do.
|
||||
|
||||
sleep := 1 * time.Microsecond // pad, so time.After doesn't send immediately
|
||||
for {
|
||||
select {
|
||||
case err := <-notifyChan:
|
||||
case err := <-caller.notify:
|
||||
return nil, err
|
||||
case s := <-ch:
|
||||
if call.slots.acquireSlot(s) {
|
||||
@@ -550,7 +558,7 @@ func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
||||
sleep = a.cfg.HotPoll
|
||||
// send a notification to launchHot()
|
||||
select {
|
||||
case call.slots.signaller <- notifyChan:
|
||||
case call.slots.signaller <- caller:
|
||||
default:
|
||||
}
|
||||
}
|
||||
@@ -671,7 +679,7 @@ func (s *hotSlot) dispatch(ctx context.Context, call *call) chan error {
|
||||
resp, err := s.container.udsClient.Do(req)
|
||||
if err != nil {
|
||||
common.Logger(ctx).WithError(err).Error("Got error from UDS socket")
|
||||
errApp <- models.NewAPIError(http.StatusBadGateway, errors.New("error receiving function response"))
|
||||
errApp <- models.ErrFunctionResponse
|
||||
return
|
||||
}
|
||||
common.Logger(ctx).WithField("resp", resp).Debug("Got resp from UDS socket")
|
||||
@@ -737,114 +745,177 @@ func newSizerRespWriter(max uint64, rw http.ResponseWriter) http.ResponseWriter
|
||||
|
||||
func (s *sizerRespWriter) Write(b []byte) (int, error) { return s.w.Write(b) }
|
||||
|
||||
func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state ContainerState) {
|
||||
// Try to queue an error to the error channel if possible.
|
||||
func tryQueueErr(err error, ch chan error) error {
|
||||
if err != nil {
|
||||
select {
|
||||
case ch <- err:
|
||||
default:
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *agent) runHot(ctx context.Context, caller slotCaller, call *call, tok ResourceToken, state ContainerState) {
|
||||
// IMPORTANT: get a context that has a child span / logger but NO timeout
|
||||
// TODO this is a 'FollowsFrom'
|
||||
ctx = common.BackgroundContext(ctx)
|
||||
ctx, span := trace.StartSpan(ctx, "agent_run_hot")
|
||||
defer span.End()
|
||||
|
||||
// IMPORTANT: evict token is deleted *after* resource token in defer statements below.
|
||||
// This ordering allows resource token to be freed first, which means once evict token
|
||||
// is deleted, eviction is considered to be completed.
|
||||
var container *container
|
||||
var cookie drivers.Cookie
|
||||
var err error
|
||||
|
||||
id := id.New().String()
|
||||
logger := logrus.WithFields(logrus.Fields{"id": id, "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "memory": call.Memory, "cpus": call.CPUs, "idle_timeout": call.IdleTimeout})
|
||||
ctx, cancel := context.WithCancel(common.WithLogger(ctx, logger))
|
||||
|
||||
udsWait := make(chan error, 1) // track UDS state and errors
|
||||
errQueue := make(chan error, 1) // errors to be reflected back to the slot queue
|
||||
|
||||
evictor := a.evictor.CreateEvictToken(call.slotHashId, call.Memory+uint64(call.TmpFsSize), uint64(call.CPUs))
|
||||
defer a.evictor.DeleteEvictToken(evictor)
|
||||
|
||||
statsUtilization(ctx, a.resources.GetUtilization())
|
||||
state.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||
|
||||
// stack unwind spelled out with strict ordering below.
|
||||
defer func() {
|
||||
// IMPORTANT: we ignore any errors due to eviction and do not reflect these to clients.
|
||||
if !evictor.isEvicted() {
|
||||
select {
|
||||
case err := <-udsWait:
|
||||
tryQueueErr(err, errQueue)
|
||||
default:
|
||||
tryQueueErr(models.ErrContainerInitFail, errQueue)
|
||||
}
|
||||
select {
|
||||
case err := <-errQueue:
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// shutdown the container and related I/O operations and go routines
|
||||
cancel()
|
||||
|
||||
// IMPORTANT: for release cookie (remove container), make sure ctx below has no timeout.
|
||||
if cookie != nil {
|
||||
cookie.Close(common.BackgroundContext(ctx))
|
||||
}
|
||||
|
||||
if container != nil {
|
||||
container.Close()
|
||||
}
|
||||
|
||||
lastState := state.GetState()
|
||||
state.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
|
||||
tok.Close() // release cpu/mem
|
||||
|
||||
// IMPORTANT: evict token is deleted *after* resource token.
|
||||
// This ordering allows resource token to be freed first, which means once evict token
|
||||
// is deleted, eviction is considered to be completed.
|
||||
a.evictor.DeleteEvictToken(evictor)
|
||||
|
||||
statsUtilization(ctx, a.resources.GetUtilization())
|
||||
if evictor.isEvicted() {
|
||||
logger.Debugf("Hot function evicted")
|
||||
statsContainerEvicted(ctx, lastState)
|
||||
}
|
||||
}()
|
||||
|
||||
defer tok.Close() // IMPORTANT: this MUST get called
|
||||
// Monitor initialization and evictability.
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case err := <-udsWait:
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
cancel()
|
||||
}
|
||||
return
|
||||
case <-ctx.Done(): // container shutdown
|
||||
return
|
||||
case <-a.shutWg.Closer(): // agent shutdown
|
||||
cancel()
|
||||
return
|
||||
case <-caller.done: // original caller disconnected or serviced by another container?
|
||||
evictor.SetEvictable(true)
|
||||
caller.done = nil // block 'caller.done' after this point
|
||||
case <-evictor.C: // eviction
|
||||
cancel()
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
state.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||
defer state.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
|
||||
container, err := newHotContainer(ctx, call, &a.cfg)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
return
|
||||
}
|
||||
defer container.Close()
|
||||
|
||||
logger := logrus.WithFields(logrus.Fields{"id": container.id, "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "memory": call.Memory, "cpus": call.CPUs, "idle_timeout": call.IdleTimeout})
|
||||
ctx = common.WithLogger(ctx, logger)
|
||||
|
||||
ctx, cancel := context.WithTimeout(common.BackgroundContext(ctx), a.cfg.HotStartTimeout)
|
||||
defer cancel()
|
||||
|
||||
cookie, err := a.driver.CreateCookie(ctx, container)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
container = newHotContainer(ctx, call, &a.cfg, id, udsWait)
|
||||
if container == nil {
|
||||
return
|
||||
}
|
||||
|
||||
// WARNING: we wait forever.
|
||||
defer cookie.Close(common.BackgroundContext(ctx))
|
||||
|
||||
shouldPull, err := cookie.ValidateImage(ctx)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
cookie, err = a.driver.CreateCookie(ctx, container)
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if shouldPull {
|
||||
needsPull, err := cookie.ValidateImage(ctx)
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if needsPull {
|
||||
ctx, cancel := context.WithTimeout(ctx, a.cfg.HotPullTimeout)
|
||||
err = cookie.PullImage(ctx)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
cancel()
|
||||
if ctx.Err() == context.DeadlineExceeded {
|
||||
err = models.ErrDockerPullTimeout
|
||||
}
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
err = cookie.CreateContainer(ctx)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
|
||||
cancel()
|
||||
ctx, shutdownContainer := context.WithCancel(common.BackgroundContext(ctx))
|
||||
defer shutdownContainer() // close this if our waiter returns, to call off slots, needs to follow cookie.Close so the cookie crumbles
|
||||
|
||||
udsAwait := make(chan error)
|
||||
// start our listener before starting the container, so we don't miss the pretty things whispered in our ears
|
||||
// make sure this thread has the shutdownContainer context in case the container exits
|
||||
go inotifyUDS(ctx, container.UDSAgentPath(), udsAwait)
|
||||
|
||||
waiter, err := cookie.Run(ctx)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer shutdownContainer() // also close if we get an agent shutdown / idle timeout
|
||||
defer cancel() // also close if we get an agent shutdown / idle timeout
|
||||
|
||||
// now we wait for the socket to be created before handing out any slots, need this
|
||||
// here in case the container dies before making the sock we need to bail
|
||||
// INIT BARRIER HERE.
|
||||
select {
|
||||
case err := <-udsAwait: // XXX(reed): need to leave a note about pairing ctx here?
|
||||
// sends a nil error if all is good, we can proceed...
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
case err := <-udsWait:
|
||||
if tryQueueErr(err, errQueue) != nil {
|
||||
return
|
||||
}
|
||||
|
||||
case <-a.shutWg.Closer(): // agent shutdown
|
||||
return
|
||||
case <-ctx.Done():
|
||||
// XXX(reed): this seems like a bad idea? why are we even handing out a
|
||||
// bad slot? shouldn't we make the client wait for a valid one and maybe
|
||||
// timeout? not in this PR, NOT TONIGHT!
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: models.ErrContainerExitedEarly})
|
||||
return
|
||||
case <-evictor.C: // eviction
|
||||
return
|
||||
case <-time.After(a.cfg.HotStartTimeout):
|
||||
tryQueueErr(models.ErrContainerInitTimeout, errQueue)
|
||||
return
|
||||
}
|
||||
|
||||
for {
|
||||
select { // make sure everything is up before trying to send slot
|
||||
case <-ctx.Done(): // container shutdown
|
||||
// Below we are rather defensive and poll on evictor/ctx
|
||||
// to reduce the likelyhood of attempting to queue a hotSlot when these
|
||||
// two cases occur.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-a.shutWg.Closer(): // server shutdown
|
||||
case <-evictor.C: // eviction
|
||||
return
|
||||
default: // ok
|
||||
default:
|
||||
}
|
||||
|
||||
slot := &hotSlot{
|
||||
@@ -867,9 +938,9 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
}
|
||||
}()
|
||||
|
||||
res := waiter.Wait(ctx)
|
||||
if res.Error() != context.Canceled {
|
||||
logger.WithError(res.Error()).Info("hot function terminated")
|
||||
runRes := waiter.Wait(ctx)
|
||||
if runRes != nil && runRes.Error() != context.Canceled {
|
||||
logger.WithError(runRes.Error()).Info("hot function terminated")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -901,52 +972,69 @@ func checkSocketDestination(filename string) error {
|
||||
|
||||
return nil
|
||||
}
|
||||
func inotifyUDS(ctx context.Context, iofsDir string, awaitUDS chan<- error) {
|
||||
// XXX(reed): I forgot how to plumb channels temporarily forgive me for this sin (inotify will timeout, this is just bad programming)
|
||||
err := inotifyAwait(ctx, iofsDir)
|
||||
if err == nil {
|
||||
err = checkSocketDestination(filepath.Join(iofsDir, udsFilename))
|
||||
}
|
||||
select {
|
||||
case awaitUDS <- err:
|
||||
case <-ctx.Done():
|
||||
}
|
||||
}
|
||||
|
||||
func inotifyAwait(ctx context.Context, iofsDir string) error {
|
||||
func inotifyAwait(ctx context.Context, iofsDir string, udsWait chan error) {
|
||||
ctx, span := trace.StartSpan(ctx, "inotify_await")
|
||||
defer span.End()
|
||||
|
||||
logger := common.Logger(ctx)
|
||||
|
||||
// Here we create the fs notify (inotify) synchronously and once that is
|
||||
// setup, then fork off our async go-routine. Basically fsnotify should be enabled
|
||||
// before we launch the container in order not to miss any events.
|
||||
fsWatcher, err := fsnotify.NewWatcher()
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting fsnotify watcher: %v", err)
|
||||
udsWait <- fmt.Errorf("error getting fsnotify watcher: %v", err)
|
||||
return
|
||||
}
|
||||
defer func() {
|
||||
if err := fsWatcher.Close(); err != nil {
|
||||
common.Logger(ctx).WithError(err).Error("Failed to close inotify watcher")
|
||||
}
|
||||
}()
|
||||
|
||||
err = fsWatcher.Add(iofsDir)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error adding iofs dir to fswatcher: %v", err)
|
||||
if err := fsWatcher.Close(); err != nil {
|
||||
logger.WithError(err).Error("Failed to close inotify watcher")
|
||||
}
|
||||
udsWait <- fmt.Errorf("error adding iofs dir to fswatcher: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
// XXX(reed): damn it would sure be nice to tell users they didn't make a uds and that's why it timed out
|
||||
return ctx.Err()
|
||||
case err := <-fsWatcher.Errors:
|
||||
return fmt.Errorf("error watching for iofs: %v", err)
|
||||
case event := <-fsWatcher.Events:
|
||||
common.Logger(ctx).WithField("event", event).Debug("fsnotify event")
|
||||
if event.Op&fsnotify.Create == fsnotify.Create && event.Name == filepath.Join(iofsDir, udsFilename) {
|
||||
// wait until the socket file is created by the container
|
||||
return nil
|
||||
go func() {
|
||||
ctx, span := trace.StartSpan(ctx, "inotify_await_poller")
|
||||
defer span.End()
|
||||
|
||||
defer func() {
|
||||
if err := fsWatcher.Close(); err != nil {
|
||||
logger.WithError(err).Error("Failed to close inotify watcher")
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case err := <-fsWatcher.Errors:
|
||||
// TODO: We do not know if these cases would be due to customer container/FDK
|
||||
// fault or some kind of service/runner issue. As conservative choice,
|
||||
// we reflect back a non API error, which means a 500 back to user.
|
||||
logger.WithError(err).Error("error watching for iofs")
|
||||
udsWait <- err
|
||||
return
|
||||
case event := <-fsWatcher.Events:
|
||||
logger.WithField("event", event).Debug("fsnotify event")
|
||||
if event.Op&fsnotify.Create == fsnotify.Create && event.Name == filepath.Join(iofsDir, udsFilename) {
|
||||
err := checkSocketDestination(filepath.Join(iofsDir, udsFilename))
|
||||
if err != nil {
|
||||
// This case is more like a bad FDK/container, so let's reflect this back to
|
||||
// clients as container init fail.
|
||||
logger.WithError(err).Error("Failed to check socket destination")
|
||||
udsWait <- models.ErrContainerInitFail
|
||||
} else {
|
||||
close(udsWait)
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// runHotReq enqueues a free slot to slot queue manager and watches various timers and the consumer until
|
||||
@@ -956,7 +1044,6 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
||||
|
||||
var err error
|
||||
isFrozen := false
|
||||
isEvictEvent := false
|
||||
|
||||
freezeTimer := time.NewTimer(a.cfg.FreezeIdle)
|
||||
idleTimer := time.NewTimer(time.Duration(call.IdleTimeout) * time.Second)
|
||||
@@ -980,7 +1067,7 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
||||
select {
|
||||
case <-s.trigger: // slot already consumed
|
||||
case <-ctx.Done(): // container shutdown
|
||||
case <-a.shutWg.Closer(): // server shutdown
|
||||
case <-a.shutWg.Closer(): // agent shutdown
|
||||
case <-idleTimer.C:
|
||||
case <-freezeTimer.C:
|
||||
if !isFrozen {
|
||||
@@ -995,8 +1082,6 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
||||
}
|
||||
continue
|
||||
case <-evictor.C:
|
||||
logger.Debug("attempting hot function eviction")
|
||||
isEvictEvent = true
|
||||
}
|
||||
break
|
||||
}
|
||||
@@ -1008,9 +1093,6 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
||||
// otherwise continue processing the request
|
||||
if call.slots.acquireSlot(s) {
|
||||
slot.Close()
|
||||
if isEvictEvent {
|
||||
statsContainerEvicted(ctx, state.GetState())
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
@@ -1058,8 +1140,24 @@ type container struct {
|
||||
}
|
||||
|
||||
//newHotContainer creates a container that can be used for multiple sequential events
|
||||
func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container, error) {
|
||||
id := id.New().String()
|
||||
func newHotContainer(ctx context.Context, call *call, cfg *Config, id string, udsWait chan error) *container {
|
||||
|
||||
var iofs iofs
|
||||
var err error
|
||||
|
||||
logger := common.Logger(ctx)
|
||||
|
||||
if cfg.IOFSEnableTmpfs {
|
||||
iofs, err = newTmpfsIOFS(ctx, cfg)
|
||||
} else {
|
||||
iofs, err = newDirectoryIOFS(ctx, cfg)
|
||||
}
|
||||
if err != nil {
|
||||
udsWait <- err
|
||||
return nil
|
||||
}
|
||||
|
||||
inotifyAwait(ctx, iofs.AgentPath(), udsWait)
|
||||
|
||||
stderr := common.NewGhostWriter()
|
||||
stdout := common.NewGhostWriter()
|
||||
@@ -1095,18 +1193,6 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
||||
stdout.Swap(newLineWriterWithBuffer(buf1, soc))
|
||||
stderr.Swap(newLineWriterWithBuffer(buf2, sec))
|
||||
|
||||
var iofs iofs
|
||||
var err error
|
||||
if cfg.IOFSEnableTmpfs {
|
||||
iofs, err = newTmpfsIOFS(ctx, cfg)
|
||||
} else {
|
||||
iofs, err = newDirectoryIOFS(ctx, cfg)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &container{
|
||||
id: id, // XXX we could just let docker generate ids...
|
||||
image: call.Image,
|
||||
@@ -1144,13 +1230,11 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
||||
for _, b := range bufs {
|
||||
bufPool.Put(b)
|
||||
}
|
||||
// iofs.Close MUST be called here or we will leak directories and/or tmpfs mounts!
|
||||
if err = iofs.Close(); err != nil {
|
||||
// Note: This is logged with the context of the container creation
|
||||
common.Logger(ctx).WithError(err).Error("Error closing IOFS")
|
||||
if err := iofs.Close(); err != nil {
|
||||
logger.WithError(err).Error("Error closing IOFS")
|
||||
}
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *container) swap(stdout, stderr io.Writer, cs *drivers.Stats) func() {
|
||||
|
||||
Reference in New Issue
Block a user