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.slots, isNew = a.slotMgr.getSlotQueue(call.slotHashId)
|
||||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
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
|
return s, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// hotLauncher is spawned in a go routine for each slot queue to monitor stats and launch hot
|
// 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,
|
// containers if needed. Upon shutdown or activity timeout, hotLauncher exits and during exit,
|
||||||
// it destroys the slot queue.
|
// 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
|
// 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
|
// whichever is longer. If in this time, there's no activity, then
|
||||||
// we destroy the hot queue.
|
// 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")
|
ctx, span := trace.StartSpan(ctx, "agent_hot_launcher")
|
||||||
defer span.End()
|
defer span.End()
|
||||||
|
|
||||||
var notifyChan chan error
|
|
||||||
|
|
||||||
for {
|
for {
|
||||||
ctx, cancel := context.WithTimeout(ctx, timeout)
|
ctx, cancel := context.WithTimeout(ctx, timeout)
|
||||||
a.checkLaunch(ctx, call, notifyChan)
|
a.checkLaunch(ctx, call, *caller)
|
||||||
notifyChan = nil
|
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case <-a.shutWg.Closer(): // server shutdown
|
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")
|
logger.Debug("Hot function launcher timed out")
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
case notifyChan = <-call.slots.signaller:
|
case caller = <-call.slots.signaller:
|
||||||
cancel()
|
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()
|
curStats := call.slots.getStats()
|
||||||
isNB := a.cfg.EnableNBResourceTracker
|
isNB := a.cfg.EnableNBResourceTracker
|
||||||
if !isNewContainerNeeded(&curStats) {
|
if !isNewContainerNeeded(&curStats) {
|
||||||
@@ -471,19 +481,19 @@ func (a *agent) checkLaunch(ctx context.Context, call *call, notifyChan chan err
|
|||||||
if tok != nil {
|
if tok != nil {
|
||||||
if tok.Error() != nil {
|
if tok.Error() != nil {
|
||||||
if tok.Error() != CapacityFull {
|
if tok.Error() != CapacityFull {
|
||||||
tryNotify(notifyChan, tok.Error())
|
tryNotify(caller.notify, tok.Error())
|
||||||
} else {
|
} else {
|
||||||
needMem, needCpu := tok.NeededCapacity()
|
needMem, needCpu := tok.NeededCapacity()
|
||||||
notifyChans = a.evictor.PerformEviction(call.slotHashId, needMem, uint64(needCpu))
|
notifyChans = a.evictor.PerformEviction(call.slotHashId, needMem, uint64(needCpu))
|
||||||
// For Non-blocking mode, if there's nothing to evict, we emit 503.
|
// For Non-blocking mode, if there's nothing to evict, we emit 503.
|
||||||
if len(notifyChans) == 0 && isNB {
|
if len(notifyChans) == 0 && isNB {
|
||||||
tryNotify(notifyChan, tok.Error())
|
tryNotify(caller.notify, tok.Error())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else if a.shutWg.AddSession(1) {
|
} else if a.shutWg.AddSession(1) {
|
||||||
go func() {
|
go func() {
|
||||||
// NOTE: runHot will not inherit the timeout from ctx (ignore timings)
|
// 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()
|
a.shutWg.DoneSession()
|
||||||
}()
|
}()
|
||||||
// early return (do not allow container state to switch to ContainerStateDone)
|
// 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
|
// 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")
|
ctx, span := trace.StartSpan(ctx, "agent_wait_hot")
|
||||||
defer span.End()
|
defer span.End()
|
||||||
|
|
||||||
@@ -519,15 +529,13 @@ func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
|||||||
|
|
||||||
ch := call.slots.startDequeuer(ctx)
|
ch := call.slots.startDequeuer(ctx)
|
||||||
|
|
||||||
notifyChan := make(chan error)
|
|
||||||
|
|
||||||
// 1) if we can get a slot immediately, grab it.
|
// 1) if we can get a slot immediately, grab it.
|
||||||
// 2) if we don't, send a signaller every x msecs until we do.
|
// 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
|
sleep := 1 * time.Microsecond // pad, so time.After doesn't send immediately
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case err := <-notifyChan:
|
case err := <-caller.notify:
|
||||||
return nil, err
|
return nil, err
|
||||||
case s := <-ch:
|
case s := <-ch:
|
||||||
if call.slots.acquireSlot(s) {
|
if call.slots.acquireSlot(s) {
|
||||||
@@ -550,7 +558,7 @@ func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
|||||||
sleep = a.cfg.HotPoll
|
sleep = a.cfg.HotPoll
|
||||||
// send a notification to launchHot()
|
// send a notification to launchHot()
|
||||||
select {
|
select {
|
||||||
case call.slots.signaller <- notifyChan:
|
case call.slots.signaller <- caller:
|
||||||
default:
|
default:
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -671,7 +679,7 @@ func (s *hotSlot) dispatch(ctx context.Context, call *call) chan error {
|
|||||||
resp, err := s.container.udsClient.Do(req)
|
resp, err := s.container.udsClient.Do(req)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
common.Logger(ctx).WithError(err).Error("Got error from UDS socket")
|
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
|
return
|
||||||
}
|
}
|
||||||
common.Logger(ctx).WithField("resp", resp).Debug("Got resp from UDS socket")
|
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 (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
|
// IMPORTANT: get a context that has a child span / logger but NO timeout
|
||||||
// TODO this is a 'FollowsFrom'
|
// TODO this is a 'FollowsFrom'
|
||||||
ctx = common.BackgroundContext(ctx)
|
ctx = common.BackgroundContext(ctx)
|
||||||
ctx, span := trace.StartSpan(ctx, "agent_run_hot")
|
ctx, span := trace.StartSpan(ctx, "agent_run_hot")
|
||||||
defer span.End()
|
defer span.End()
|
||||||
|
|
||||||
// IMPORTANT: evict token is deleted *after* resource token in defer statements below.
|
var container *container
|
||||||
// This ordering allows resource token to be freed first, which means once evict token
|
var cookie drivers.Cookie
|
||||||
// is deleted, eviction is considered to be completed.
|
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))
|
evictor := a.evictor.CreateEvictToken(call.slotHashId, call.Memory+uint64(call.TmpFsSize), uint64(call.CPUs))
|
||||||
defer a.evictor.DeleteEvictToken(evictor)
|
|
||||||
|
|
||||||
statsUtilization(ctx, a.resources.GetUtilization())
|
statsUtilization(ctx, a.resources.GetUtilization())
|
||||||
|
state.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||||
|
|
||||||
|
// stack unwind spelled out with strict ordering below.
|
||||||
defer func() {
|
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())
|
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)
|
container = newHotContainer(ctx, call, &a.cfg, id, udsWait)
|
||||||
defer state.UpdateState(ctx, ContainerStateDone, call.slots)
|
if container == nil {
|
||||||
|
|
||||||
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})
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// WARNING: we wait forever.
|
cookie, err = a.driver.CreateCookie(ctx, container)
|
||||||
defer cookie.Close(common.BackgroundContext(ctx))
|
if tryQueueErr(err, errQueue) != nil {
|
||||||
|
|
||||||
shouldPull, err := cookie.ValidateImage(ctx)
|
|
||||||
if err != nil {
|
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
|
||||||
return
|
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)
|
err = cookie.PullImage(ctx)
|
||||||
if err != nil {
|
cancel()
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
if ctx.Err() == context.DeadlineExceeded {
|
||||||
|
err = models.ErrDockerPullTimeout
|
||||||
|
}
|
||||||
|
if tryQueueErr(err, errQueue) != nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
err = cookie.CreateContainer(ctx)
|
err = cookie.CreateContainer(ctx)
|
||||||
if err != nil {
|
if tryQueueErr(err, errQueue) != nil {
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
|
||||||
return
|
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)
|
waiter, err := cookie.Run(ctx)
|
||||||
if err != nil {
|
if tryQueueErr(err, errQueue) != nil {
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
go func() {
|
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
|
// INIT BARRIER HERE.
|
||||||
// here in case the container dies before making the sock we need to bail
|
|
||||||
select {
|
select {
|
||||||
case err := <-udsAwait: // XXX(reed): need to leave a note about pairing ctx here?
|
case err := <-udsWait:
|
||||||
// sends a nil error if all is good, we can proceed...
|
if tryQueueErr(err, errQueue) != nil {
|
||||||
if err != nil {
|
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
case <-a.shutWg.Closer(): // agent shutdown
|
||||||
|
return
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
// XXX(reed): this seems like a bad idea? why are we even handing out a
|
return
|
||||||
// bad slot? shouldn't we make the client wait for a valid one and maybe
|
case <-evictor.C: // eviction
|
||||||
// timeout? not in this PR, NOT TONIGHT!
|
return
|
||||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: models.ErrContainerExitedEarly})
|
case <-time.After(a.cfg.HotStartTimeout):
|
||||||
|
tryQueueErr(models.ErrContainerInitTimeout, errQueue)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
for {
|
for {
|
||||||
select { // make sure everything is up before trying to send slot
|
// Below we are rather defensive and poll on evictor/ctx
|
||||||
case <-ctx.Done(): // container shutdown
|
// to reduce the likelyhood of attempting to queue a hotSlot when these
|
||||||
|
// two cases occur.
|
||||||
|
select {
|
||||||
|
case <-ctx.Done():
|
||||||
return
|
return
|
||||||
case <-a.shutWg.Closer(): // server shutdown
|
case <-evictor.C: // eviction
|
||||||
return
|
return
|
||||||
default: // ok
|
default:
|
||||||
}
|
}
|
||||||
|
|
||||||
slot := &hotSlot{
|
slot := &hotSlot{
|
||||||
@@ -867,9 +938,9 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
|||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
res := waiter.Wait(ctx)
|
runRes := waiter.Wait(ctx)
|
||||||
if res.Error() != context.Canceled {
|
if runRes != nil && runRes.Error() != context.Canceled {
|
||||||
logger.WithError(res.Error()).Info("hot function terminated")
|
logger.WithError(runRes.Error()).Info("hot function terminated")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -901,52 +972,69 @@ func checkSocketDestination(filename string) error {
|
|||||||
|
|
||||||
return nil
|
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")
|
ctx, span := trace.StartSpan(ctx, "inotify_await")
|
||||||
defer span.End()
|
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()
|
fsWatcher, err := fsnotify.NewWatcher()
|
||||||
if err != nil {
|
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)
|
err = fsWatcher.Add(iofsDir)
|
||||||
if err != nil {
|
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 {
|
go func() {
|
||||||
select {
|
ctx, span := trace.StartSpan(ctx, "inotify_await_poller")
|
||||||
case <-ctx.Done():
|
defer span.End()
|
||||||
// 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()
|
defer func() {
|
||||||
case err := <-fsWatcher.Errors:
|
if err := fsWatcher.Close(); err != nil {
|
||||||
return fmt.Errorf("error watching for iofs: %v", err)
|
logger.WithError(err).Error("Failed to close inotify watcher")
|
||||||
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
|
for {
|
||||||
return nil
|
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
|
// 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
|
var err error
|
||||||
isFrozen := false
|
isFrozen := false
|
||||||
isEvictEvent := false
|
|
||||||
|
|
||||||
freezeTimer := time.NewTimer(a.cfg.FreezeIdle)
|
freezeTimer := time.NewTimer(a.cfg.FreezeIdle)
|
||||||
idleTimer := time.NewTimer(time.Duration(call.IdleTimeout) * time.Second)
|
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 {
|
select {
|
||||||
case <-s.trigger: // slot already consumed
|
case <-s.trigger: // slot already consumed
|
||||||
case <-ctx.Done(): // container shutdown
|
case <-ctx.Done(): // container shutdown
|
||||||
case <-a.shutWg.Closer(): // server shutdown
|
case <-a.shutWg.Closer(): // agent shutdown
|
||||||
case <-idleTimer.C:
|
case <-idleTimer.C:
|
||||||
case <-freezeTimer.C:
|
case <-freezeTimer.C:
|
||||||
if !isFrozen {
|
if !isFrozen {
|
||||||
@@ -995,8 +1082,6 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
|||||||
}
|
}
|
||||||
continue
|
continue
|
||||||
case <-evictor.C:
|
case <-evictor.C:
|
||||||
logger.Debug("attempting hot function eviction")
|
|
||||||
isEvictEvent = true
|
|
||||||
}
|
}
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
@@ -1008,9 +1093,6 @@ func (a *agent) runHotReq(ctx context.Context, call *call, state ContainerState,
|
|||||||
// otherwise continue processing the request
|
// otherwise continue processing the request
|
||||||
if call.slots.acquireSlot(s) {
|
if call.slots.acquireSlot(s) {
|
||||||
slot.Close()
|
slot.Close()
|
||||||
if isEvictEvent {
|
|
||||||
statsContainerEvicted(ctx, state.GetState())
|
|
||||||
}
|
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -1058,8 +1140,24 @@ type container struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
//newHotContainer creates a container that can be used for multiple sequential events
|
//newHotContainer creates a container that can be used for multiple sequential events
|
||||||
func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container, error) {
|
func newHotContainer(ctx context.Context, call *call, cfg *Config, id string, udsWait chan error) *container {
|
||||||
id := id.New().String()
|
|
||||||
|
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()
|
stderr := common.NewGhostWriter()
|
||||||
stdout := common.NewGhostWriter()
|
stdout := common.NewGhostWriter()
|
||||||
@@ -1095,18 +1193,6 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
|||||||
stdout.Swap(newLineWriterWithBuffer(buf1, soc))
|
stdout.Swap(newLineWriterWithBuffer(buf1, soc))
|
||||||
stderr.Swap(newLineWriterWithBuffer(buf2, sec))
|
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{
|
return &container{
|
||||||
id: id, // XXX we could just let docker generate ids...
|
id: id, // XXX we could just let docker generate ids...
|
||||||
image: call.Image,
|
image: call.Image,
|
||||||
@@ -1144,13 +1230,11 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
|||||||
for _, b := range bufs {
|
for _, b := range bufs {
|
||||||
bufPool.Put(b)
|
bufPool.Put(b)
|
||||||
}
|
}
|
||||||
// iofs.Close MUST be called here or we will leak directories and/or tmpfs mounts!
|
if err := iofs.Close(); err != nil {
|
||||||
if err = iofs.Close(); err != nil {
|
logger.WithError(err).Error("Error closing IOFS")
|
||||||
// Note: This is logged with the context of the container creation
|
|
||||||
common.Logger(ctx).WithError(err).Error("Error closing IOFS")
|
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
}, nil
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *container) swap(stdout, stderr io.Writer, cs *drivers.Stats) func() {
|
func (c *container) swap(stdout, stderr io.Writer, cs *drivers.Stats) func() {
|
||||||
|
|||||||
369
api/agent/agent_evict_test.go
Normal file
369
api/agent/agent_evict_test.go
Normal file
@@ -0,0 +1,369 @@
|
|||||||
|
package agent
|
||||||
|
|
||||||
|
import (
|
||||||
|
"bytes"
|
||||||
|
"context"
|
||||||
|
"net/http"
|
||||||
|
"net/http/httptest"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
"sync"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
_ "github.com/fnproject/fn/api/agent/drivers/docker"
|
||||||
|
"github.com/fnproject/fn/api/id"
|
||||||
|
"github.com/fnproject/fn/api/logs"
|
||||||
|
"github.com/fnproject/fn/api/models"
|
||||||
|
"github.com/fnproject/fn/api/mqs"
|
||||||
|
)
|
||||||
|
|
||||||
|
// create a simple non-blocking agent. Non-blocking does not queue, so it's
|
||||||
|
// easier to test and see if evictions took place.
|
||||||
|
func getAgent() (Agent, error) {
|
||||||
|
ls := logs.NewMock()
|
||||||
|
cfg, err := NewConfig()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// 160MB memory
|
||||||
|
cfg.EnableNBResourceTracker = true
|
||||||
|
cfg.HotPoll = 20
|
||||||
|
cfg.MaxTotalMemory = 160 * 1024 * 1024
|
||||||
|
cfg.HotPullTimeout = time.Duration(10000) * time.Millisecond
|
||||||
|
cfg.HotStartTimeout = time.Duration(10000) * time.Millisecond
|
||||||
|
|
||||||
|
a := New(NewDirectCallDataAccess(ls, new(mqs.Mock)), WithConfig(cfg))
|
||||||
|
return a, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func getHungDocker() (*httptest.Server, func()) {
|
||||||
|
hung, cancel := context.WithCancel(context.Background())
|
||||||
|
|
||||||
|
srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||||
|
// version check seem to have a sane timeout in docker, let's serve this, then stop
|
||||||
|
if r.URL.String() == "/v2/" {
|
||||||
|
w.WriteHeader(200)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
<-hung.Done()
|
||||||
|
}))
|
||||||
|
|
||||||
|
closer := func() {
|
||||||
|
cancel()
|
||||||
|
srv.Close()
|
||||||
|
}
|
||||||
|
|
||||||
|
return srv, closer
|
||||||
|
}
|
||||||
|
|
||||||
|
func getApp() *models.App {
|
||||||
|
return &models.App{ID: id.New().String()}
|
||||||
|
}
|
||||||
|
|
||||||
|
func getFn(initDelayMsecs int) *models.Fn {
|
||||||
|
fn := &models.Fn{
|
||||||
|
ID: id.New().String(),
|
||||||
|
Image: "fnproject/fn-test-utils",
|
||||||
|
ResourceConfig: models.ResourceConfig{
|
||||||
|
Timeout: 10,
|
||||||
|
IdleTimeout: 60,
|
||||||
|
Memory: 128, // only 1 fit in 160MB
|
||||||
|
},
|
||||||
|
}
|
||||||
|
if initDelayMsecs > 0 {
|
||||||
|
fn.Config = models.Config{"ENABLE_INIT_DELAY_MSEC": strconv.FormatUint(uint64(initDelayMsecs), 10)}
|
||||||
|
}
|
||||||
|
return fn
|
||||||
|
}
|
||||||
|
|
||||||
|
// simple GetCall/Submit combo.
|
||||||
|
func execFn(input string, fn *models.Fn, app *models.App, a Agent, tmsec int) error {
|
||||||
|
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), time.Duration(tmsec)*time.Millisecond)
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
url := "http://127.0.0.1:8080/invoke/" + fn.ID
|
||||||
|
|
||||||
|
req, err := http.NewRequest("GET", url, &dummyReader{Reader: strings.NewReader(input)})
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
req = req.WithContext(ctx)
|
||||||
|
|
||||||
|
var out bytes.Buffer
|
||||||
|
callI, err := a.GetCall(FromHTTPFnRequest(app, fn, req), WithWriter(&out))
|
||||||
|
if err != nil {
|
||||||
|
if ctx.Err() != nil {
|
||||||
|
return ctx.Err()
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
err = a.Submit(callI)
|
||||||
|
if ctx.Err() != nil {
|
||||||
|
return ctx.Err()
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBadContainer1(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
fn.Config = models.Config{"ENABLE_INIT_EXIT": "0"}
|
||||||
|
|
||||||
|
err = execFn(`{"sleepTime": 8000}`, fn, getApp(), a, 20000)
|
||||||
|
if err != models.ErrContainerInitFail {
|
||||||
|
t.Fatalf("submit unexpected error! %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBadContainer2(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
fn.Config = models.Config{"ENABLE_INIT_EXIT": "0", "ENABLE_INIT_DELAY_MSEC": "200"}
|
||||||
|
|
||||||
|
err = execFn(`{"sleepTime": 8000}`, fn, getApp(), a, 20000)
|
||||||
|
if err != models.ErrContainerInitFail {
|
||||||
|
t.Fatalf("submit unexpected error! %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBadContainer3(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
|
||||||
|
err = execFn(`{"isCrash": true }`, fn, getApp(), a, 20000)
|
||||||
|
if err != models.ErrFunctionResponse {
|
||||||
|
t.Fatalf("submit unexpected error! %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestBadContainer4(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
|
||||||
|
err = execFn(`{"isExit": true, "exitCode": 0 }`, fn, getApp(), a, 20000)
|
||||||
|
if err != models.ErrFunctionResponse {
|
||||||
|
t.Fatalf("submit unexpected error! %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Eviction will NOT take place since the first container is busy
|
||||||
|
func TestPlainNoEvict(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
err := execFn(`{"sleepTime": 8000}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("submit should not error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(3000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != models.ErrCallTimeoutServerBusy {
|
||||||
|
t.Fatalf("unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
|
// Eviction will take place since the first container is idle
|
||||||
|
func TestPlainDoEvict(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("submit should not error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(3000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("submit should not error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHungFDKNoEvict(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(11000), getApp(), a, 20000)
|
||||||
|
if err != models.ErrContainerInitTimeout {
|
||||||
|
t.Fatalf("submit unexpected error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(3000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != models.ErrCallTimeoutServerBusy {
|
||||||
|
t.Fatalf("unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestHungFDKDoEvict(t *testing.T) {
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(11000), getApp(), a, 1000)
|
||||||
|
if err != context.DeadlineExceeded {
|
||||||
|
t.Fatalf("submit expected context.DeadlineExceeded %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(3000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("submit should not error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDockerPullHungDoEvict(t *testing.T) {
|
||||||
|
dockerSrv, dockerCancel := getHungDocker()
|
||||||
|
defer dockerCancel()
|
||||||
|
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
fn.Image = strings.TrimPrefix(dockerSrv.URL, "http://") + "/" + fn.Image
|
||||||
|
|
||||||
|
err := execFn(`{"sleepTime": 0}`, fn, getApp(), a, 1000)
|
||||||
|
if err != context.DeadlineExceeded {
|
||||||
|
t.Fatalf("submit expected context.DeadlineExceeded %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(2000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("submit should not error! %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDockerPullHungNoEvict(t *testing.T) {
|
||||||
|
dockerSrv, dockerCancel := getHungDocker()
|
||||||
|
defer dockerCancel()
|
||||||
|
|
||||||
|
a, err := getAgent()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("cannot create agent")
|
||||||
|
}
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(2)
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
|
||||||
|
fn := getFn(0)
|
||||||
|
fn.Image = strings.TrimPrefix(dockerSrv.URL, "http://") + "/" + fn.Image
|
||||||
|
|
||||||
|
err := execFn(`{"sleepTime": 0}`, fn, getApp(), a, 20000)
|
||||||
|
if err != models.ErrDockerPullTimeout {
|
||||||
|
t.Fatalf("unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
defer wg.Done()
|
||||||
|
time.Sleep(3000 * time.Millisecond)
|
||||||
|
err := execFn(`{"sleepTime": 0}`, getFn(0), getApp(), a, 20000)
|
||||||
|
if err != models.ErrCallTimeoutServerBusy {
|
||||||
|
t.Fatalf("unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
wg.Wait()
|
||||||
|
|
||||||
|
}
|
||||||
@@ -7,9 +7,13 @@ import (
|
|||||||
"encoding/json"
|
"encoding/json"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
|
"io/ioutil"
|
||||||
"math"
|
"math"
|
||||||
|
"net"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/http/httptest"
|
"net/http/httptest"
|
||||||
|
"os"
|
||||||
|
"path/filepath"
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
@@ -21,11 +25,8 @@ import (
|
|||||||
"github.com/fnproject/fn/api/logs"
|
"github.com/fnproject/fn/api/logs"
|
||||||
"github.com/fnproject/fn/api/models"
|
"github.com/fnproject/fn/api/models"
|
||||||
"github.com/fnproject/fn/api/mqs"
|
"github.com/fnproject/fn/api/mqs"
|
||||||
|
|
||||||
"github.com/sirupsen/logrus"
|
"github.com/sirupsen/logrus"
|
||||||
"io/ioutil"
|
|
||||||
"net"
|
|
||||||
"os"
|
|
||||||
"path/filepath"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
@@ -436,6 +437,48 @@ type dummyReader struct {
|
|||||||
io.Reader
|
io.Reader
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestHungFDK(t *testing.T) {
|
||||||
|
app := &models.App{ID: "app_id"}
|
||||||
|
fn := &models.Fn{
|
||||||
|
ID: "fn_id",
|
||||||
|
Image: "fnproject/fn-test-utils",
|
||||||
|
Config: models.Config{"ENABLE_INIT_DELAY_MSEC": "5000"},
|
||||||
|
ResourceConfig: models.ResourceConfig{
|
||||||
|
Timeout: 5,
|
||||||
|
IdleTimeout: 10,
|
||||||
|
Memory: 128,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
url := "http://127.0.0.1:8080/invoke/" + fn.ID
|
||||||
|
|
||||||
|
ls := logs.NewMock()
|
||||||
|
cfg, err := NewConfig()
|
||||||
|
cfg.MaxDockerRetries = 1
|
||||||
|
cfg.HotStartTimeout = time.Duration(3) * time.Second
|
||||||
|
a := New(NewDirectCallDataAccess(ls, new(mqs.Mock)), WithConfig(cfg))
|
||||||
|
defer checkClose(t, a)
|
||||||
|
|
||||||
|
req, err := http.NewRequest("GET", url, &dummyReader{Reader: strings.NewReader(`{}`)})
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal("unexpected error building request", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
var out bytes.Buffer
|
||||||
|
callI, err := a.GetCall(FromHTTPFnRequest(app, fn, req), WithWriter(&out))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
err = a.Submit(callI)
|
||||||
|
if err == nil {
|
||||||
|
t.Fatal("submit should error!")
|
||||||
|
}
|
||||||
|
if err != models.ErrContainerInitTimeout {
|
||||||
|
t.Fatalf("unexpected error %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestDockerPullHungRepo(t *testing.T) {
|
func TestDockerPullHungRepo(t *testing.T) {
|
||||||
hung, cancel := context.WithCancel(context.Background())
|
hung, cancel := context.WithCancel(context.Background())
|
||||||
garbageServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
garbageServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||||
@@ -467,7 +510,7 @@ func TestDockerPullHungRepo(t *testing.T) {
|
|||||||
ls := logs.NewMock()
|
ls := logs.NewMock()
|
||||||
cfg, err := NewConfig()
|
cfg, err := NewConfig()
|
||||||
cfg.MaxDockerRetries = 1
|
cfg.MaxDockerRetries = 1
|
||||||
cfg.HotStartTimeout = time.Duration(5) * time.Second
|
cfg.HotPullTimeout = time.Duration(5) * time.Second
|
||||||
a := New(NewDirectCallDataAccess(ls, new(mqs.Mock)), WithConfig(cfg))
|
a := New(NewDirectCallDataAccess(ls, new(mqs.Mock)), WithConfig(cfg))
|
||||||
defer checkClose(t, a)
|
defer checkClose(t, a)
|
||||||
|
|
||||||
@@ -486,8 +529,7 @@ func TestDockerPullHungRepo(t *testing.T) {
|
|||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatal("submit should error!")
|
t.Fatal("submit should error!")
|
||||||
}
|
}
|
||||||
errS := err.Error()
|
if err != models.ErrDockerPullTimeout {
|
||||||
if !strings.HasPrefix(errS, "Failed to pull image ") || !strings.Contains(errS, "context deadline exceeded") {
|
|
||||||
t.Fatalf("unexpected error %v", err)
|
t.Fatalf("unexpected error %v", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -535,7 +577,7 @@ func TestDockerPullBadRepo(t *testing.T) {
|
|||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatal("submit should error!")
|
t.Fatal("submit should error!")
|
||||||
}
|
}
|
||||||
if !strings.HasPrefix(err.Error(), "Failed to pull image ") {
|
if !models.IsAPIError(err) || !strings.HasPrefix(err.Error(), "Failed to pull image ") {
|
||||||
t.Fatalf("unexpected error %v", err)
|
t.Fatalf("unexpected error %v", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -1219,8 +1261,11 @@ func TestDockerAuthExtn(t *testing.T) {
|
|||||||
|
|
||||||
ctx := context.TODO()
|
ctx := context.TODO()
|
||||||
|
|
||||||
c, err := newHotContainer(ctx, call, cfg)
|
errC := make(chan error, 10)
|
||||||
if err != nil {
|
|
||||||
|
c := newHotContainer(ctx, call, cfg, id.New().String(), errC)
|
||||||
|
if c == nil {
|
||||||
|
err := <-errC
|
||||||
t.Fatal("got unexpected err: ", err)
|
t.Fatal("got unexpected err: ", err)
|
||||||
}
|
}
|
||||||
da, err := c.DockerAuth()
|
da, err := c.DockerAuth()
|
||||||
@@ -1236,8 +1281,9 @@ func TestDockerAuthExtn(t *testing.T) {
|
|||||||
extn["FN_REGISTRY_TOKEN"] = "TestRegistryToken"
|
extn["FN_REGISTRY_TOKEN"] = "TestRegistryToken"
|
||||||
call.extensions = extn
|
call.extensions = extn
|
||||||
|
|
||||||
c, err = newHotContainer(ctx, call, cfg)
|
c = newHotContainer(ctx, call, cfg, id.New().String(), errC)
|
||||||
if err != nil {
|
if c == nil {
|
||||||
|
err := <-errC
|
||||||
t.Fatal("got unexpected err: ", err)
|
t.Fatal("got unexpected err: ", err)
|
||||||
}
|
}
|
||||||
da, err = c.DockerAuth()
|
da, err = c.DockerAuth()
|
||||||
|
|||||||
@@ -16,6 +16,7 @@ type Config struct {
|
|||||||
FreezeIdle time.Duration `json:"freeze_idle_msecs"`
|
FreezeIdle time.Duration `json:"freeze_idle_msecs"`
|
||||||
HotPoll time.Duration `json:"hot_poll_msecs"`
|
HotPoll time.Duration `json:"hot_poll_msecs"`
|
||||||
HotLauncherTimeout time.Duration `json:"hot_launcher_timeout_msecs"`
|
HotLauncherTimeout time.Duration `json:"hot_launcher_timeout_msecs"`
|
||||||
|
HotPullTimeout time.Duration `json:"hot_pull_timeout_msecs"`
|
||||||
HotStartTimeout time.Duration `json:"hot_start_timeout_msecs"`
|
HotStartTimeout time.Duration `json:"hot_start_timeout_msecs"`
|
||||||
AsyncChewPoll time.Duration `json:"async_chew_poll_msecs"`
|
AsyncChewPoll time.Duration `json:"async_chew_poll_msecs"`
|
||||||
DetachedHeadRoom time.Duration `json:"detached_head_room_msecs"`
|
DetachedHeadRoom time.Duration `json:"detached_head_room_msecs"`
|
||||||
@@ -52,7 +53,9 @@ const (
|
|||||||
EnvHotPoll = "FN_HOT_POLL_MSECS"
|
EnvHotPoll = "FN_HOT_POLL_MSECS"
|
||||||
// EnvHotLauncherTimeout is the timeout for a hot container queue to persist if idle
|
// EnvHotLauncherTimeout is the timeout for a hot container queue to persist if idle
|
||||||
EnvHotLauncherTimeout = "FN_HOT_LAUNCHER_TIMEOUT_MSECS"
|
EnvHotLauncherTimeout = "FN_HOT_LAUNCHER_TIMEOUT_MSECS"
|
||||||
// EnvHotStartTimeout is the timeout for a hot container to become available for use including docker-pull
|
// EnvHotStartTimeout is the timeout for a hot container to be created including docker-pull
|
||||||
|
EnvHotPullTimeout = "FN_HOT_PULL_TIMEOUT_MSECS"
|
||||||
|
// EnvHotStartTimeout is the timeout for a hot container to become available for use for requests after EnvHotStartTimeout
|
||||||
EnvHotStartTimeout = "FN_HOT_START_TIMEOUT_MSECS"
|
EnvHotStartTimeout = "FN_HOT_START_TIMEOUT_MSECS"
|
||||||
// EnvAsyncChewPoll is the interval to poll the queue that contains async function invocations
|
// EnvAsyncChewPoll is the interval to poll the queue that contains async function invocations
|
||||||
EnvAsyncChewPoll = "FN_ASYNC_CHEW_POLL_MSECS"
|
EnvAsyncChewPoll = "FN_ASYNC_CHEW_POLL_MSECS"
|
||||||
@@ -130,7 +133,8 @@ func NewConfig() (*Config, error) {
|
|||||||
err = setEnvMsecs(err, EnvFreezeIdle, &cfg.FreezeIdle, 50*time.Millisecond)
|
err = setEnvMsecs(err, EnvFreezeIdle, &cfg.FreezeIdle, 50*time.Millisecond)
|
||||||
err = setEnvMsecs(err, EnvHotPoll, &cfg.HotPoll, DefaultHotPoll)
|
err = setEnvMsecs(err, EnvHotPoll, &cfg.HotPoll, DefaultHotPoll)
|
||||||
err = setEnvMsecs(err, EnvHotLauncherTimeout, &cfg.HotLauncherTimeout, time.Duration(60)*time.Minute)
|
err = setEnvMsecs(err, EnvHotLauncherTimeout, &cfg.HotLauncherTimeout, time.Duration(60)*time.Minute)
|
||||||
err = setEnvMsecs(err, EnvHotStartTimeout, &cfg.HotStartTimeout, time.Duration(10)*time.Minute)
|
err = setEnvMsecs(err, EnvHotPullTimeout, &cfg.HotPullTimeout, time.Duration(10)*time.Minute)
|
||||||
|
err = setEnvMsecs(err, EnvHotStartTimeout, &cfg.HotStartTimeout, time.Duration(10)*time.Second)
|
||||||
err = setEnvMsecs(err, EnvAsyncChewPoll, &cfg.AsyncChewPoll, time.Duration(60)*time.Second)
|
err = setEnvMsecs(err, EnvAsyncChewPoll, &cfg.AsyncChewPoll, time.Duration(60)*time.Second)
|
||||||
err = setEnvMsecs(err, EnvDetachedHeadroom, &cfg.DetachedHeadRoom, time.Duration(360)*time.Second)
|
err = setEnvMsecs(err, EnvDetachedHeadroom, &cfg.DetachedHeadRoom, time.Duration(360)*time.Second)
|
||||||
err = setEnvUint(err, EnvMaxResponseSize, &cfg.MaxResponseSize)
|
err = setEnvUint(err, EnvMaxResponseSize, &cfg.MaxResponseSize)
|
||||||
|
|||||||
@@ -43,6 +43,11 @@ type slotToken struct {
|
|||||||
isBusy uint32
|
isBusy uint32
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type slotCaller struct {
|
||||||
|
notify chan error // notification to caller
|
||||||
|
done <-chan struct{} // caller done
|
||||||
|
}
|
||||||
|
|
||||||
// LIFO queue that exposes input/output channels along
|
// LIFO queue that exposes input/output channels along
|
||||||
// with runner/waiter tracking for agent
|
// with runner/waiter tracking for agent
|
||||||
type slotQueue struct {
|
type slotQueue struct {
|
||||||
@@ -50,7 +55,7 @@ type slotQueue struct {
|
|||||||
cond *sync.Cond
|
cond *sync.Cond
|
||||||
slots []*slotToken
|
slots []*slotToken
|
||||||
nextId uint64
|
nextId uint64
|
||||||
signaller chan chan error
|
signaller chan *slotCaller
|
||||||
statsLock sync.Mutex // protects stats below
|
statsLock sync.Mutex // protects stats below
|
||||||
stats slotQueueStats
|
stats slotQueueStats
|
||||||
}
|
}
|
||||||
@@ -67,7 +72,7 @@ func NewSlotQueue(key string) *slotQueue {
|
|||||||
key: key,
|
key: key,
|
||||||
cond: sync.NewCond(new(sync.Mutex)),
|
cond: sync.NewCond(new(sync.Mutex)),
|
||||||
slots: make([]*slotToken, 0),
|
slots: make([]*slotToken, 0),
|
||||||
signaller: make(chan chan error, 1),
|
signaller: make(chan *slotCaller, 1),
|
||||||
}
|
}
|
||||||
|
|
||||||
return obj
|
return obj
|
||||||
|
|||||||
@@ -31,7 +31,14 @@ var (
|
|||||||
code: http.StatusServiceUnavailable,
|
code: http.StatusServiceUnavailable,
|
||||||
error: errors.New("Timed out - server too busy"),
|
error: errors.New("Timed out - server too busy"),
|
||||||
}
|
}
|
||||||
|
ErrDockerPullTimeout = err{
|
||||||
|
code: http.StatusServiceUnavailable,
|
||||||
|
error: errors.New("Docker pull timed out"),
|
||||||
|
}
|
||||||
|
ErrContainerInitTimeout = err{
|
||||||
|
code: http.StatusServiceUnavailable,
|
||||||
|
error: errors.New("Container initialization timed out, please ensure you are using the latest fdk / format and check the logs"),
|
||||||
|
}
|
||||||
ErrUnsupportedMediaType = err{
|
ErrUnsupportedMediaType = err{
|
||||||
code: http.StatusUnsupportedMediaType,
|
code: http.StatusUnsupportedMediaType,
|
||||||
error: errors.New("Content Type not supported")}
|
error: errors.New("Content Type not supported")}
|
||||||
@@ -122,6 +129,10 @@ var (
|
|||||||
code: http.StatusBadGateway,
|
code: http.StatusBadGateway,
|
||||||
error: fmt.Errorf("function response too large"),
|
error: fmt.Errorf("function response too large"),
|
||||||
}
|
}
|
||||||
|
ErrFunctionResponse = err{
|
||||||
|
code: http.StatusBadGateway,
|
||||||
|
error: fmt.Errorf("error receiving function response"),
|
||||||
|
}
|
||||||
ErrRequestContentTooBig = err{
|
ErrRequestContentTooBig = err{
|
||||||
code: http.StatusRequestEntityTooLarge,
|
code: http.StatusRequestEntityTooLarge,
|
||||||
error: fmt.Errorf("Request content too large"),
|
error: fmt.Errorf("Request content too large"),
|
||||||
@@ -150,7 +161,6 @@ var (
|
|||||||
code: http.StatusTooManyRequests,
|
code: http.StatusTooManyRequests,
|
||||||
error: errors.New("Too many requests submitted"),
|
error: errors.New("Too many requests submitted"),
|
||||||
}
|
}
|
||||||
|
|
||||||
ErrAsyncUnsupported = err{
|
ErrAsyncUnsupported = err{
|
||||||
code: http.StatusBadRequest,
|
code: http.StatusBadRequest,
|
||||||
error: errors.New("Async functions are not supported on this server"),
|
error: errors.New("Async functions are not supported on this server"),
|
||||||
@@ -166,10 +176,9 @@ var (
|
|||||||
error: errors.New("Unable to find the call handle"),
|
error: errors.New("Unable to find the call handle"),
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO consider removal. see rationale at uses, or remove if none.
|
ErrContainerInitFail = err{
|
||||||
ErrContainerExitedEarly = err{
|
|
||||||
code: http.StatusBadGateway,
|
code: http.StatusBadGateway,
|
||||||
error: errors.New("container exited early, please ensure you are using the latest fdk / format and check the logs"),
|
error: errors.New("container failed to initialize, please ensure you are using the latest fdk / format and check the logs"),
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|||||||
@@ -210,7 +210,7 @@ func TestFnInvokeRunnerExecution(t *testing.T) {
|
|||||||
{"/invoke/http_stream_fn_id", multiLog, "POST", http.StatusOK, nil, "", multiLogExpectHot},
|
{"/invoke/http_stream_fn_id", multiLog, "POST", http.StatusOK, nil, "", multiLogExpectHot},
|
||||||
|
|
||||||
// TODO consider removing this, see comment above the image
|
// TODO consider removing this, see comment above the image
|
||||||
{"/invoke/fail_fn", ok, "POST", http.StatusBadGateway, nil, "container exited early, please ensure", nil},
|
{"/invoke/fail_fn", ok, "POST", http.StatusBadGateway, nil, "container failed to initialize", nil},
|
||||||
}
|
}
|
||||||
|
|
||||||
callIds := make([]string, len(testCases))
|
callIds := make([]string, len(testCases))
|
||||||
|
|||||||
@@ -40,6 +40,10 @@ type AppRequest struct {
|
|||||||
IsDebug bool `json:"isDebug,omitempty"`
|
IsDebug bool `json:"isDebug,omitempty"`
|
||||||
// simulate crash
|
// simulate crash
|
||||||
IsCrash bool `json:"isCrash,omitempty"`
|
IsCrash bool `json:"isCrash,omitempty"`
|
||||||
|
// abrupt exit code
|
||||||
|
ExitCode int `json:"exitCode,omitempty"`
|
||||||
|
// enable abrupt exit
|
||||||
|
IsExit bool `json:"isExit,omitempty"`
|
||||||
// shutdown UDS after request
|
// shutdown UDS after request
|
||||||
IsShutdown bool `json:"isShutdown,omitempty"`
|
IsShutdown bool `json:"isShutdown,omitempty"`
|
||||||
// read a file from disk
|
// read a file from disk
|
||||||
@@ -232,6 +236,11 @@ func processRequest(ctx context.Context, in io.Reader) (*AppRequest, *AppRespons
|
|||||||
log.Fatalln("Crash requested")
|
log.Fatalln("Crash requested")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if request.IsExit {
|
||||||
|
log.Printf("Exit requested %+v", request.ExitCode)
|
||||||
|
os.Exit(request.ExitCode)
|
||||||
|
}
|
||||||
|
|
||||||
if request.ExpectHeaders != nil {
|
if request.ExpectHeaders != nil {
|
||||||
for name, header := range request.ExpectHeaders {
|
for name, header := range request.ExpectHeaders {
|
||||||
if h2 := fnctx.Header().Get(name); header[0] != h2 {
|
if h2 := fnctx.Header().Get(name); header[0] != h2 {
|
||||||
@@ -316,6 +325,15 @@ func main() {
|
|||||||
time.Sleep(time.Millisecond * time.Duration(delay))
|
time.Sleep(time.Millisecond * time.Duration(delay))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if initExit := os.Getenv("ENABLE_INIT_EXIT"); initExit != "" {
|
||||||
|
log.Printf("Container start exit %v", initExit)
|
||||||
|
exitCode, err := strconv.ParseInt(initExit, 10, 64)
|
||||||
|
if err != nil {
|
||||||
|
log.Fatalf("cannot parse ENABLE_INIT_EXIT %v", err)
|
||||||
|
}
|
||||||
|
os.Exit(int(exitCode))
|
||||||
|
}
|
||||||
|
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
GlobCancel = cancel
|
GlobCancel = cancel
|
||||||
fdk.HandleContext(ctx, fdk.HandlerFunc(AppHandler)) // XXX(reed): can extract & instrument
|
fdk.HandleContext(ctx, fdk.HandlerFunc(AppHandler)) // XXX(reed): can extract & instrument
|
||||||
|
|||||||
Reference in New Issue
Block a user