mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
fn: better slot/container/request state tracking (#719)
* fn: better slot/container/request state tracking
This commit is contained in:
@@ -180,9 +180,32 @@ func (a *agent) Submit(callI Call) error {
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *agent) startStateTrackers(ctx context.Context, call *call) {
|
||||
|
||||
if !protocol.IsStreamable(protocol.Protocol(call.Format)) {
|
||||
// For cold containers, we track the container state in call
|
||||
call.containerState = NewContainerState()
|
||||
}
|
||||
|
||||
call.requestState = NewRequestState()
|
||||
}
|
||||
|
||||
func (a *agent) endStateTrackers(ctx context.Context, call *call) {
|
||||
|
||||
call.requestState.UpdateState(ctx, RequestStateDone, call.slots)
|
||||
|
||||
// For cold containers, we are done with the container.
|
||||
if call.containerState != nil {
|
||||
call.containerState.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
}
|
||||
}
|
||||
|
||||
func (a *agent) submit(ctx context.Context, call *call) error {
|
||||
a.stats.Enqueue(ctx, call.AppName, call.Path)
|
||||
|
||||
a.startStateTrackers(ctx, call)
|
||||
defer a.endStateTrackers(ctx, call)
|
||||
|
||||
slot, err := a.getSlot(ctx, call)
|
||||
if err != nil {
|
||||
a.handleStatsDequeue(ctx, call, err)
|
||||
@@ -226,10 +249,10 @@ func transformTimeout(e error, isRetriable bool) error {
|
||||
func (a *agent) handleStatsDequeue(ctx context.Context, call *call, err error) {
|
||||
if err == context.DeadlineExceeded {
|
||||
a.stats.Dequeue(ctx, call.AppName, call.Path)
|
||||
a.stats.IncrementTooBusy(ctx)
|
||||
IncrementTooBusy(ctx)
|
||||
} else {
|
||||
a.stats.DequeueAndFail(ctx, call.AppName, call.Path)
|
||||
a.stats.IncrementErrors(ctx)
|
||||
IncrementErrors(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -243,9 +266,9 @@ func (a *agent) handleStatsEnd(ctx context.Context, call *call, err error) {
|
||||
a.stats.Failed(ctx, call.AppName, call.Path)
|
||||
// increment the timeout or errors count, as appropriate
|
||||
if err == context.DeadlineExceeded {
|
||||
a.stats.IncrementTimedout(ctx)
|
||||
IncrementTimedout(ctx)
|
||||
} else {
|
||||
a.stats.IncrementErrors(ctx)
|
||||
IncrementErrors(ctx)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -282,22 +305,19 @@ func (a *agent) getSlot(ctx context.Context, call *call) (Slot, error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_get_slot")
|
||||
defer span.Finish()
|
||||
|
||||
isHot := protocol.IsStreamable(protocol.Protocol(call.Format))
|
||||
if isHot {
|
||||
start := time.Now()
|
||||
|
||||
if protocol.IsStreamable(protocol.Protocol(call.Format)) {
|
||||
// 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)
|
||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
||||
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
|
||||
}
|
||||
|
||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
||||
return a.launchCold(ctx, call)
|
||||
}
|
||||
|
||||
@@ -351,18 +371,24 @@ func (a *agent) checkLaunch(ctx context.Context, call *call) {
|
||||
if !isNeeded {
|
||||
return
|
||||
}
|
||||
common.Logger(ctx).WithFields(logrus.Fields{"currentStats": curStats, "isNeeded": isNeeded}).Info("Hot function launcher starting hot container")
|
||||
|
||||
state := NewContainerState()
|
||||
state.UpdateState(ctx, ContainerStateWait, call.slots)
|
||||
|
||||
common.Logger(ctx).WithFields(logrus.Fields{"currentStats": call.slots.getStats(), "isNeeded": isNeeded}).Info("Hot function launcher starting hot container")
|
||||
|
||||
select {
|
||||
case tok := <-a.resources.GetResourceToken(ctx, call.Memory, uint64(call.CPUs), isAsync):
|
||||
a.wg.Add(1) // add waiter in this thread
|
||||
go func() {
|
||||
// NOTE: runHot will not inherit the timeout from ctx (ignore timings)
|
||||
a.runHot(ctx, call, tok)
|
||||
a.runHot(ctx, call, tok, state)
|
||||
a.wg.Done()
|
||||
}()
|
||||
case <-ctx.Done(): // timeout
|
||||
state.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
case <-a.shutdown: // server shutdown
|
||||
state.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -418,6 +444,8 @@ func (a *agent) launchCold(ctx context.Context, call *call) (Slot, error) {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_launch_cold")
|
||||
defer span.Finish()
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateWait, call.slots)
|
||||
|
||||
select {
|
||||
case tok := <-a.resources.GetResourceToken(ctx, call.Memory, uint64(call.CPUs), isAsync):
|
||||
go a.prepCold(ctx, call, tok, ch)
|
||||
@@ -453,6 +481,9 @@ func (s *coldSlot) exec(ctx context.Context, call *call) error {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_cold_exec")
|
||||
defer span.Finish()
|
||||
|
||||
call.requestState.UpdateState(ctx, RequestStateExec, call.slots)
|
||||
call.containerState.UpdateState(ctx, ContainerStateBusy, call.slots)
|
||||
|
||||
waiter, err := s.cookie.Run(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
@@ -504,12 +535,11 @@ func (s *hotSlot) exec(ctx context.Context, call *call) error {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_hot_exec")
|
||||
defer span.Finish()
|
||||
|
||||
call.requestState.UpdateState(ctx, RequestStateExec, call.slots)
|
||||
|
||||
// link the container id and id in the logs [for us!]
|
||||
common.Logger(ctx).WithField("container_id", s.container.id).Info("starting call")
|
||||
|
||||
start := time.Now()
|
||||
defer func() { call.slots.recordLatency(SlotQueueRunner, uint64(time.Now().Sub(start).Seconds()*1000)) }()
|
||||
|
||||
// swap in the new stderr logger & stat accumulator
|
||||
oldStderr := s.container.swap(call.stderr, &call.Stats)
|
||||
defer s.container.swap(oldStderr, nil) // once we're done, swap out in this scope to prevent races
|
||||
@@ -542,6 +572,8 @@ func (a *agent) prepCold(ctx context.Context, call *call, tok ResourceToken, ch
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, "agent_prep_cold")
|
||||
defer span.Finish()
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||
|
||||
// add additional headers to the config to shove everything into env vars for cold
|
||||
for k, v := range call.Headers {
|
||||
if !specialHeader(k) {
|
||||
@@ -568,6 +600,9 @@ func (a *agent) prepCold(ctx context.Context, call *call, tok ResourceToken, ch
|
||||
// pull & create container before we return a slot, so as to be friendly
|
||||
// about timing out if this takes a while...
|
||||
cookie, err := a.driver.Prepare(ctx, container)
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateIdle, call.slots)
|
||||
|
||||
slot := &coldSlot{cookie, tok, err}
|
||||
select {
|
||||
case ch <- slot:
|
||||
@@ -576,7 +611,7 @@ func (a *agent) prepCold(ctx context.Context, call *call, tok ResourceToken, ch
|
||||
}
|
||||
}
|
||||
|
||||
func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken) {
|
||||
func (a *agent) runHot(ctx context.Context, 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 = opentracing.ContextWithSpan(context.Background(), opentracing.SpanFromContext(ctx))
|
||||
@@ -590,8 +625,8 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken) {
|
||||
|
||||
proto := protocol.New(protocol.Protocol(call.Format), stdinWrite, stdoutRead)
|
||||
|
||||
start := time.Now()
|
||||
call.slots.enterState(SlotQueueStarter)
|
||||
state.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||
defer state.UpdateState(ctx, ContainerStateDone, call.slots)
|
||||
|
||||
cid := id.New().String()
|
||||
|
||||
@@ -617,7 +652,6 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken) {
|
||||
|
||||
cookie, err := a.driver.Prepare(ctx, container)
|
||||
if err != nil {
|
||||
call.slots.exitStateWithLatency(SlotQueueStarter, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), err: err})
|
||||
return
|
||||
}
|
||||
@@ -625,15 +659,12 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken) {
|
||||
|
||||
waiter, err := cookie.Run(ctx)
|
||||
if err != nil {
|
||||
call.slots.exitStateWithLatency(SlotQueueStarter, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), err: err})
|
||||
return
|
||||
}
|
||||
|
||||
// container is running
|
||||
call.slots.enterState(SlotQueueRunner)
|
||||
call.slots.exitStateWithLatency(SlotQueueStarter, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
defer call.slots.exitState(SlotQueueRunner)
|
||||
state.UpdateState(ctx, ContainerStateIdle, call.slots)
|
||||
|
||||
// buffered, in case someone has slot when waiter returns but isn't yet listening
|
||||
errC := make(chan error, 1)
|
||||
@@ -653,30 +684,27 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken) {
|
||||
}
|
||||
|
||||
done := make(chan struct{})
|
||||
start := time.Now()
|
||||
call.slots.enterState(SlotQueueIdle)
|
||||
state.UpdateState(ctx, ContainerStateIdle, call.slots)
|
||||
s := call.slots.queueSlot(&hotSlot{done, proto, errC, container, nil})
|
||||
|
||||
select {
|
||||
case <-s.trigger:
|
||||
call.slots.exitStateWithLatency(SlotQueueIdle, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
case <-time.After(time.Duration(call.IdleTimeout) * time.Second):
|
||||
if call.slots.ejectSlot(s) {
|
||||
call.slots.exitStateWithLatency(SlotQueueIdle, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
logger.Info("Canceling inactive hot function")
|
||||
return
|
||||
}
|
||||
case <-ctx.Done(): // container shutdown
|
||||
if call.slots.ejectSlot(s) {
|
||||
call.slots.exitStateWithLatency(SlotQueueIdle, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
return
|
||||
}
|
||||
case <-a.shutdown: // server shutdown
|
||||
if call.slots.ejectSlot(s) {
|
||||
call.slots.exitStateWithLatency(SlotQueueIdle, uint64(time.Now().Sub(start).Seconds()*1000))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
state.UpdateState(ctx, ContainerStateBusy, call.slots)
|
||||
// IMPORTANT: if we fail to eject the slot, it means that a consumer
|
||||
// just dequeued this and acquired the slot. In other words, we were
|
||||
// late in ejectSlots(), so we have to execute this request in this
|
||||
|
||||
Reference in New Issue
Block a user