mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
death to format (#1281)
* get rid of old format stuff, utils usage, fix up for fdk2.0 interface * pure agent format removal, TODO remove format field, fix up all tests * shitter's clogged * fix agent tests * start rolling through server tests * tests compile, some failures * remove json / content type detection on invoke/httptrigger, fix up tests * remove hello, fixup system tests the fucking status checker test just hangs and it's testing that it doesn't work so the test passes but the test doesn't pass fuck life it's not worth it * fix migration * meh * make dbhelper shut up about dbhelpers not being used * move fail status at least into main thread, jfc * fix status call to have FN_LISTENER also turns off the stdout/stderr blocking between calls, because it's impossible to debug without that (without syslog), now that stdout and stderr go to the same place (either to host stderr or nowhere) and isn't used for function output this shouldn't be a big fuss really * remove stdin * cleanup/remind: fixed bug where watcher would leak if container dies first * silence system-test logs until fail, fix datastore tests postgres does weird things with constraints when renaming tables, took the easy way out system-tests were loud as fuck and made you download a circleci text file of the logs, made them only yell when they goof * fix fdk-go dep for test image. fun * fix swagger and remove test about format * update all the gopkg files * add back FN_FORMAT for fdks that assert things. pfft * add useful error for functions that exit this error is really confounding because containers can exit for all manner of reason, we're just guessing that this is the most likely cause for now, and this error message should very likely change or be removed from the client path anyway (context.Canceled wasn't all that useful either, but anyway, I'd been hunting for this... so found it). added a test to avoid being publicly shamed for 1 line commits (beware...).
This commit is contained in:
@@ -15,7 +15,6 @@ import (
|
||||
"path/filepath"
|
||||
|
||||
"github.com/fnproject/fn/api/agent/drivers"
|
||||
"github.com/fnproject/fn/api/agent/protocol"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/id"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
@@ -37,8 +36,6 @@ import (
|
||||
// to be much more robust. now we're at least running it if we delete the msg,
|
||||
// but we may never store info about that execution so still broked (if fn
|
||||
// dies). need coordination w/ db.
|
||||
// TODO if a cold call times out but container is created but hasn't replied, could
|
||||
// end up that the client doesn't get a reply until long after the timeout (b/c of container removal, async it?)
|
||||
// TODO if async would store requests (or interchange format) it would be slick, but
|
||||
// if we're going to store full calls in db maybe we should only queue pointers to ids?
|
||||
// TODO examine cases where hot can't start a container and the user would never see an error
|
||||
@@ -53,7 +50,7 @@ import (
|
||||
// those calls, it also exposes a 'safe' shutdown mechanism via its Close method.
|
||||
// Agent has a few roles:
|
||||
// * manage the memory pool for a given server
|
||||
// * manage the container lifecycle for calls (hot+cold)
|
||||
// * manage the container lifecycle for calls
|
||||
// * execute calls against containers
|
||||
// * invoke Start and End for each call appropriately
|
||||
// * check the mq for any async calls, and submit them
|
||||
@@ -61,19 +58,16 @@ import (
|
||||
// Overview:
|
||||
// Upon submission of a call, Agent will start the call's timeout timer
|
||||
// immediately. If the call is hot, Agent will attempt to find an active hot
|
||||
// container for that route, and if necessary launch another container. Cold
|
||||
// calls will launch one container each. Cold calls will get container input
|
||||
// and output directly, whereas hot calls will be able to read/write directly
|
||||
// from/to a pipe in a container via Dispatch. If it's necessary to launch a
|
||||
// container, first an attempt will be made to try to reserve the ram required
|
||||
// while waiting for any hot 'slot' to become available [if applicable]. If
|
||||
// there is an error launching the container, an error will be returned
|
||||
// provided the call has not yet timed out or found another hot 'slot' to
|
||||
// execute in [if applicable]. call.Start will be called immediately before
|
||||
// starting a container, if cold (i.e. after pulling), or immediately before
|
||||
// sending any input, if hot. call.End will be called regardless of the
|
||||
// timeout timer's status if the call was executed, and that error returned may
|
||||
// be returned from Submit.
|
||||
// container for that route, and if necessary launch another container. calls
|
||||
// will be able to read/write directly from/to a socket in the container. If
|
||||
// it's necessary to launch a container, first an attempt will be made to try
|
||||
// to reserve the ram required while waiting for any hot 'slot' to become
|
||||
// available [if applicable]. If there is an error launching the container, an
|
||||
// error will be returned provided the call has not yet timed out or found
|
||||
// another hot 'slot' to execute in [if applicable]. call.Start will be called
|
||||
// immediately before sending any input to a container. call.End will be called
|
||||
// regardless of the timeout timer's status if the call was executed, and that
|
||||
// error returned may be returned from Submit.
|
||||
type Agent interface {
|
||||
// GetCall will return a Call that is executable by the Agent, which
|
||||
// can be built via various CallOpt's provided to the method.
|
||||
@@ -266,23 +260,11 @@ func (a *agent) Submit(callI Call) error {
|
||||
}
|
||||
|
||||
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 {
|
||||
@@ -345,9 +327,9 @@ func (a *agent) handleCallEnd(ctx context.Context, call *call, slot Slot, err er
|
||||
return err
|
||||
}
|
||||
|
||||
// getSlot returns a Slot (or error) for the request to run. Depending on hot/cold
|
||||
// request type, this may launch a new container or wait for other containers to become idle
|
||||
// or it may wait for resources to become available to launch a new container.
|
||||
// getSlot returns a Slot (or error) for the request to run. This will wait
|
||||
// for other containers to become idle or it may wait for resources to become
|
||||
// available to launch a new container.
|
||||
func (a *agent) getSlot(ctx context.Context, call *call) (Slot, error) {
|
||||
if call.Type == models.TypeAsync {
|
||||
// *) for async, slot deadline is also call.Timeout. This is because we would like to
|
||||
@@ -364,25 +346,20 @@ func (a *agent) getSlot(ctx context.Context, call *call) (Slot, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "agent_get_slot")
|
||||
defer span.End()
|
||||
|
||||
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
|
||||
// For hot requests, we use a long lived slot queue, which we use to manage hot containers
|
||||
var isNew bool
|
||||
|
||||
if call.slotHashId == "" {
|
||||
call.slotHashId = getSlotQueueKey(call)
|
||||
}
|
||||
|
||||
call.slots, isNew = a.slotMgr.getSlotQueue(call.slotHashId)
|
||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
||||
if isNew {
|
||||
go a.hotLauncher(ctx, call)
|
||||
}
|
||||
s, err := a.waitHot(ctx, call)
|
||||
return s, err
|
||||
if call.slotHashId == "" {
|
||||
call.slotHashId = getSlotQueueKey(call)
|
||||
}
|
||||
|
||||
call.slots, isNew = a.slotMgr.getSlotQueue(call.slotHashId)
|
||||
call.requestState.UpdateState(ctx, RequestStateWait, call.slots)
|
||||
return a.launchCold(ctx, call)
|
||||
if isNew {
|
||||
go a.hotLauncher(ctx, call)
|
||||
}
|
||||
s, err := a.waitHot(ctx, call)
|
||||
return s, err
|
||||
}
|
||||
|
||||
// hotLauncher is spawned in a go routine for each slot queue to monitor stats and launch hot
|
||||
@@ -575,86 +552,6 @@ func (a *agent) waitHot(ctx context.Context, call *call) (Slot, error) {
|
||||
}
|
||||
}
|
||||
|
||||
// launchCold waits for necessary resources to launch a new container, then
|
||||
// returns the slot for that new container to run the request on.
|
||||
func (a *agent) launchCold(ctx context.Context, call *call) (Slot, error) {
|
||||
isNB := a.cfg.EnableNBResourceTracker
|
||||
|
||||
ch := make(chan Slot)
|
||||
|
||||
ctx, span := trace.StartSpan(ctx, "agent_launch_cold")
|
||||
defer span.End()
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateWait, call.slots)
|
||||
|
||||
mem := call.Memory + uint64(call.TmpFsSize)
|
||||
|
||||
select {
|
||||
case tok := <-a.resources.GetResourceToken(ctx, mem, call.CPUs, isNB):
|
||||
if tok.Error() != nil {
|
||||
return nil, tok.Error()
|
||||
}
|
||||
|
||||
go a.prepCold(ctx, call, tok, ch)
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
|
||||
// wait for launch err or a slot to open up
|
||||
select {
|
||||
case s := <-ch:
|
||||
if s.Error() != nil {
|
||||
s.Close()
|
||||
return nil, s.Error()
|
||||
}
|
||||
return s, nil
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
}
|
||||
|
||||
// implements Slot
|
||||
type coldSlot struct {
|
||||
cookie drivers.Cookie
|
||||
tok ResourceToken
|
||||
closer func()
|
||||
fatalErr error
|
||||
}
|
||||
|
||||
func (s *coldSlot) Error() error {
|
||||
return s.fatalErr
|
||||
}
|
||||
|
||||
func (s *coldSlot) exec(ctx context.Context, call *call) error {
|
||||
ctx, span := trace.StartSpan(ctx, "agent_cold_exec")
|
||||
defer span.End()
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
res := waiter.Wait(ctx)
|
||||
if res.Error() != nil {
|
||||
// check for call error (oom/exit) and beam it up
|
||||
return res.Error()
|
||||
}
|
||||
|
||||
// nil or timed out
|
||||
return ctx.Err()
|
||||
}
|
||||
|
||||
func (s *coldSlot) Close() error {
|
||||
if s.closer != nil {
|
||||
s.closer()
|
||||
s.closer = nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// implements Slot
|
||||
type hotSlot struct {
|
||||
done chan struct{} // signal we are done with slot
|
||||
@@ -699,12 +596,7 @@ func (s *hotSlot) exec(ctx context.Context, call *call) error {
|
||||
|
||||
call.req = call.req.WithContext(ctx) // TODO this is funny biz reed is bad
|
||||
|
||||
var errApp chan error
|
||||
if call.Format == models.FormatHTTPStream {
|
||||
errApp = s.dispatch(ctx, call)
|
||||
} else { // TODO remove this block one glorious day
|
||||
errApp = s.dispatchOldFormats(ctx, call)
|
||||
}
|
||||
errApp := s.dispatch(ctx, call)
|
||||
|
||||
select {
|
||||
case err := <-s.errC: // error from container
|
||||
@@ -714,10 +606,6 @@ func (s *hotSlot) exec(ctx context.Context, call *call) error {
|
||||
if err != nil {
|
||||
if models.IsAPIError(err) {
|
||||
s.trySetError(err)
|
||||
} else if err == protocol.ErrExcessData {
|
||||
s.trySetError(err)
|
||||
// suppress excess data error, but do shutdown the container
|
||||
return nil
|
||||
}
|
||||
}
|
||||
return err
|
||||
@@ -756,15 +644,11 @@ func callToHTTPRequest(ctx context.Context, call *call) *http.Request {
|
||||
}
|
||||
}
|
||||
|
||||
//req.Header.Set("FN_DEADLINE", ci.Deadline().String())
|
||||
// TODO(occ) : fix compatidupes when FDKs are updated
|
||||
req.Header.Set("Fn-Call-Id", call.ID)
|
||||
req.Header.Set("FN_CALL_ID", call.ID)
|
||||
deadline, ok := ctx.Deadline()
|
||||
if ok {
|
||||
deadlineStr := deadline.Format(time.RFC3339)
|
||||
req.Header.Set("Fn-Deadline", deadlineStr)
|
||||
req.Header.Set("FN_DEADLINE", deadlineStr)
|
||||
}
|
||||
|
||||
return req
|
||||
@@ -781,7 +665,7 @@ func (s *hotSlot) dispatch(ctx context.Context, call *call) chan error {
|
||||
|
||||
// TODO it's possible we can get rid of this (after getting rid of logs API) - may need for call id/debug mode still
|
||||
// TODO there's a timeout race for swapping this back if the container doesn't get killed for timing out, and don't you forget it
|
||||
swapBack := s.container.swap(nil, call.stderr, call.stderr, &call.Stats)
|
||||
swapBack := s.container.swap(call.stderr, call.stderr, &call.Stats)
|
||||
defer swapBack()
|
||||
|
||||
req := callToHTTPRequest(ctx, call)
|
||||
@@ -811,11 +695,10 @@ func (s *hotSlot) dispatch(ctx context.Context, call *call) chan error {
|
||||
return errApp
|
||||
}
|
||||
|
||||
// XXX(reed): dupe code in http proto (which will die...)
|
||||
func writeResp(max uint64, resp *http.Response, w io.Writer) error {
|
||||
rw, ok := w.(http.ResponseWriter)
|
||||
if !ok {
|
||||
w = common.NewClampWriter(rw, max, models.ErrFunctionResponseTooBig)
|
||||
w = common.NewClampWriter(w, max, models.ErrFunctionResponseTooBig)
|
||||
return resp.Write(w)
|
||||
}
|
||||
|
||||
@@ -855,110 +738,6 @@ func newSizerRespWriter(max uint64, rw http.ResponseWriter) http.ResponseWriter
|
||||
|
||||
func (s *sizerRespWriter) Write(b []byte) (int, error) { return s.w.Write(b) }
|
||||
|
||||
// TODO remove
|
||||
func (s *hotSlot) dispatchOldFormats(ctx context.Context, call *call) chan error {
|
||||
|
||||
errApp := make(chan error, 1)
|
||||
go func() {
|
||||
// XXX(reed): this may be liable to leave the pipes fucked up if dispatch times out, eg
|
||||
// we may need ye ole close() func to put the Close()/swapBack() in from the caller
|
||||
|
||||
// swap in fresh pipes & stat accumulator to not interlace with other calls that used this slot [and timed out]
|
||||
stdinRead, stdinWrite := io.Pipe()
|
||||
stdoutRead, stdoutWritePipe := io.Pipe()
|
||||
defer stdinRead.Close()
|
||||
defer stdoutWritePipe.Close()
|
||||
|
||||
// NOTE: stderr is limited separately (though line writer is vulnerable to attack?)
|
||||
// limit the bytes allowed to be written to the stdout pipe, which handles any
|
||||
// buffering overflows (json to a string, http to a buffer, etc)
|
||||
stdoutWrite := common.NewClampWriter(stdoutWritePipe, s.cfg.MaxResponseSize, models.ErrFunctionResponseTooBig)
|
||||
|
||||
swapBack := s.container.swap(stdinRead, stdoutWrite, call.stderr, &call.Stats)
|
||||
defer swapBack() // NOTE: it's important this runs before the pipes are closed.
|
||||
|
||||
// TODO this should get killed completely
|
||||
// TODO we could alternatively dial in and use the conn as stdin/stdout for an interim solution
|
||||
// XXX(reed): ^^^ do we need that for the cloud event dance ????
|
||||
proto := protocol.New(protocol.Protocol(call.Format), stdinWrite, stdoutRead)
|
||||
ci := protocol.NewCallInfo(call.IsCloudEvent, call.Call, call.req)
|
||||
errApp <- proto.Dispatch(ctx, ci, call.w)
|
||||
}()
|
||||
|
||||
return errApp
|
||||
}
|
||||
|
||||
func (a *agent) prepCold(ctx context.Context, call *call, tok ResourceToken, ch chan Slot) {
|
||||
ctx, span := trace.StartSpan(ctx, "agent_prep_cold")
|
||||
defer span.End()
|
||||
statsUtilization(ctx, a.resources.GetUtilization())
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateStart, call.slots)
|
||||
|
||||
deadline := time.Now().Add(time.Duration(call.Timeout) * time.Second)
|
||||
|
||||
// add Fn-specific information to the config to shove everything into env vars for cold
|
||||
call.Config["FN_DEADLINE"] = common.DateTime(deadline).String()
|
||||
call.Config["FN_METHOD"] = call.Model().Method
|
||||
call.Config["FN_REQUEST_URL"] = call.Model().URL
|
||||
call.Config["FN_CALL_ID"] = call.Model().ID
|
||||
|
||||
// User headers are prefixed with FN_HEADER and shoved in the env vars too
|
||||
for k, v := range call.Headers {
|
||||
k = "FN_HEADER_" + k
|
||||
call.Config[k] = strings.Join(v, ", ")
|
||||
}
|
||||
|
||||
container := &container{
|
||||
id: id.New().String(), // XXX we could just let docker generate ids...
|
||||
image: call.Image,
|
||||
env: map[string]string(call.Config),
|
||||
extensions: call.extensions,
|
||||
memory: call.Memory,
|
||||
cpus: uint64(call.CPUs),
|
||||
fsSize: a.cfg.MaxFsSize,
|
||||
iofs: &noopIOFS{},
|
||||
timeout: time.Duration(call.Timeout) * time.Second, // this is unnecessary, but in case removal fails...
|
||||
logCfg: drivers.LoggerConfig{
|
||||
URL: strings.TrimSpace(call.SyslogURL),
|
||||
Tags: []drivers.LoggerTag{
|
||||
{Name: "app_id", Value: call.AppID},
|
||||
{Name: "fn_id", Value: call.FnID},
|
||||
},
|
||||
},
|
||||
stdin: call.req.Body,
|
||||
stdout: common.NewClampWriter(call.w, a.cfg.MaxResponseSize, models.ErrFunctionResponseTooBig),
|
||||
stderr: call.stderr,
|
||||
stats: &call.Stats,
|
||||
}
|
||||
|
||||
cookie, err := a.driver.CreateCookie(ctx, container)
|
||||
if err == nil {
|
||||
// pull & create container before we return a slot, so as to be friendly
|
||||
// about timing out if this takes a while...
|
||||
err = a.driver.PrepareCookie(ctx, cookie)
|
||||
}
|
||||
|
||||
call.containerState.UpdateState(ctx, ContainerStateIdle, call.slots)
|
||||
|
||||
closer := func() {
|
||||
if cookie != nil {
|
||||
cookie.Close(ctx)
|
||||
}
|
||||
if tok != nil {
|
||||
tok.Close()
|
||||
}
|
||||
statsUtilization(ctx, a.resources.GetUtilization())
|
||||
}
|
||||
|
||||
slot := &coldSlot{cookie: cookie, tok: tok, closer: closer, fatalErr: err}
|
||||
select {
|
||||
case ch <- slot:
|
||||
case <-ctx.Done():
|
||||
slot.Close()
|
||||
}
|
||||
}
|
||||
|
||||
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'
|
||||
@@ -989,30 +768,9 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
}
|
||||
defer container.Close()
|
||||
|
||||
// NOTE: soon this isn't assigned in a branch...
|
||||
var udsClient http.Client
|
||||
udsAwait := make(chan error)
|
||||
if call.Format == models.FormatHTTPStream {
|
||||
// start our listener before starting the container, so we don't miss the pretty things whispered in our ears
|
||||
go inotifyUDS(ctx, container.UDSAgentPath(), udsAwait)
|
||||
|
||||
udsClient = http.Client{
|
||||
Transport: &http.Transport{
|
||||
MaxIdleConns: 1,
|
||||
MaxIdleConnsPerHost: 1,
|
||||
// XXX(reed): other settings ?
|
||||
IdleConnTimeout: 1 * time.Second,
|
||||
DialContext: func(ctx context.Context, _, _ string) (net.Conn, error) {
|
||||
var d net.Dialer
|
||||
return d.DialContext(ctx, "unix", filepath.Join(container.UDSAgentPath(), udsFilename))
|
||||
},
|
||||
},
|
||||
}
|
||||
} else {
|
||||
close(udsAwait) // XXX(reed): short case first / kill this
|
||||
}
|
||||
|
||||
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, "format": call.Format, "idle_timeout": call.IdleTimeout})
|
||||
// XXX(reed): we need to timeout the cookie create / prepare since docker client doesn't have timeout anymore,
|
||||
// and handle cookie close having a timed out context it still needs to delete the thing. fun stuff
|
||||
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)
|
||||
|
||||
cookie, err := a.driver.CreateCookie(ctx, container)
|
||||
@@ -1029,6 +787,27 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
return
|
||||
}
|
||||
|
||||
ctx, shutdownContainer := context.WithCancel(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)
|
||||
|
||||
udsClient := http.Client{
|
||||
Transport: &http.Transport{
|
||||
MaxIdleConns: 1,
|
||||
MaxIdleConnsPerHost: 1,
|
||||
// XXX(reed): other settings ?
|
||||
IdleConnTimeout: 1 * time.Second,
|
||||
DialContext: func(ctx context.Context, _, _ string) (net.Conn, error) {
|
||||
var d net.Dialer
|
||||
return d.DialContext(ctx, "unix", filepath.Join(container.UDSAgentPath(), udsFilename))
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
waiter, err := cookie.Run(ctx)
|
||||
if err != nil {
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: err})
|
||||
@@ -1038,8 +817,6 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
// buffered, in case someone has slot when waiter returns but isn't yet listening
|
||||
errC := make(chan error, 1)
|
||||
|
||||
ctx, shutdownContainer := context.WithCancel(ctx)
|
||||
defer shutdownContainer() // close this if our waiter returns, to call off slots
|
||||
go func() {
|
||||
defer shutdownContainer() // also close if we get an agent shutdown / idle timeout
|
||||
|
||||
@@ -1054,7 +831,10 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
}
|
||||
|
||||
case <-ctx.Done():
|
||||
call.slots.queueSlot(&hotSlot{done: make(chan struct{}), fatalErr: ctx.Err()})
|
||||
// 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
|
||||
}
|
||||
|
||||
@@ -1090,8 +870,16 @@ func (a *agent) runHot(ctx context.Context, call *call, tok ResourceToken, state
|
||||
}()
|
||||
|
||||
res := waiter.Wait(ctx)
|
||||
if res.Error() != nil {
|
||||
errC <- res.Error() // TODO: race condition, no guaranteed delivery fix this...
|
||||
if err := res.Error(); err != nil {
|
||||
if err == context.Canceled {
|
||||
// TODO consider removal after format biz mellows out (t-minus 30 days)
|
||||
// we can turn this into a user visible error (for now) if the container
|
||||
// exits because they are using the wrong format, say, and try to guess/
|
||||
// help them out for now. context.Canceled only comes from container exit.
|
||||
errC <- models.ErrContainerExitedEarly
|
||||
} else {
|
||||
errC <- err // TODO: race condition, no guaranteed delivery fix this... we can't really, if nobody is there it's working as intended for idle timeout?
|
||||
}
|
||||
}
|
||||
if res.Error() != context.Canceled {
|
||||
logger.WithError(res.Error()).Info("hot function terminated")
|
||||
@@ -1167,7 +955,6 @@ func inotifyAwait(ctx context.Context, iofsDir string) error {
|
||||
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
|
||||
}
|
||||
@@ -1266,11 +1053,9 @@ type container struct {
|
||||
fsSize uint64
|
||||
tmpFsSize uint64
|
||||
iofs iofs
|
||||
timeout time.Duration // cold only (superfluous, but in case)
|
||||
logCfg drivers.LoggerConfig
|
||||
close func()
|
||||
|
||||
stdin io.Reader
|
||||
stdout io.Writer
|
||||
stderr io.Writer
|
||||
|
||||
@@ -1281,65 +1066,52 @@ 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) {
|
||||
// if freezer is enabled, be consistent with freezer behavior and
|
||||
// block stdout and stderr between calls.
|
||||
isBlockIdleIO := MaxMsDisabled != cfg.FreezeIdle
|
||||
|
||||
id := id.New().String()
|
||||
|
||||
stdin := common.NewGhostReader()
|
||||
stderr := common.NewGhostWriter()
|
||||
stdout := common.NewGhostWriter()
|
||||
|
||||
// for use if no freezer (or we ever make up our minds)
|
||||
var bufs []*bytes.Buffer
|
||||
|
||||
// when not processing a request, do we block IO?
|
||||
if !isBlockIdleIO {
|
||||
// IMPORTANT: we are not operating on a TTY allocated container. This means, stderr and stdout are multiplexed
|
||||
// from the same stream internally via docker using a multiplexing protocol. Therefore, stderr/stdout *BOTH*
|
||||
// have to be read or *BOTH* blocked consistently. In other words, we cannot block one and continue
|
||||
// reading from the other one without risking head-of-line blocking.
|
||||
// IMPORTANT: we are not operating on a TTY allocated container. This means, stderr and stdout are multiplexed
|
||||
// from the same stream internally via docker using a multiplexing protocol. Therefore, stderr/stdout *BOTH*
|
||||
// have to be read or *BOTH* blocked consistently. In other words, we cannot block one and continue
|
||||
// reading from the other one without risking head-of-line blocking.
|
||||
|
||||
// wrap the syslog and debug loggers in the same (respective) line writer
|
||||
// syslog complete chain for this (from top):
|
||||
// stderr -> line writer
|
||||
// wrap the syslog and debug loggers in the same (respective) line writer
|
||||
// stderr -> line writer
|
||||
|
||||
// TODO(reed): I guess this is worth it
|
||||
// TODO(reed): there's a bug here where the between writers could have
|
||||
// bytes in there, get swapped for real stdout/stderr, come back and write
|
||||
// bytes in and the bytes are [really] stale. I played with fixing this
|
||||
// and mostly came to the conclusion that life is meaningless.
|
||||
buf1 := bufPool.Get().(*bytes.Buffer)
|
||||
buf2 := bufPool.Get().(*bytes.Buffer)
|
||||
bufs = []*bytes.Buffer{buf1, buf2}
|
||||
// TODO(reed): there's a bug here where the between writers could have
|
||||
// bytes in there, get swapped for real stdout/stderr, come back and write
|
||||
// bytes in and the bytes are [really] stale. I played with fixing this
|
||||
// and mostly came to the conclusion that life is meaningless.
|
||||
// TODO(reed): we should let the syslog driver pick this up really but our
|
||||
// default story sucks there
|
||||
buf1 := bufPool.Get().(*bytes.Buffer)
|
||||
buf2 := bufPool.Get().(*bytes.Buffer)
|
||||
bufs = []*bytes.Buffer{buf1, buf2}
|
||||
|
||||
soc := &nopCloser{&logWriter{
|
||||
logrus.WithFields(logrus.Fields{"tag": "stdout", "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "container_id": id}),
|
||||
}}
|
||||
sec := &nopCloser{&logWriter{
|
||||
logrus.WithFields(logrus.Fields{"tag": "stderr", "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "container_id": id}),
|
||||
}}
|
||||
soc := &nopCloser{&logWriter{
|
||||
logrus.WithFields(logrus.Fields{"tag": "stdout", "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "container_id": id}),
|
||||
}}
|
||||
sec := &nopCloser{&logWriter{
|
||||
logrus.WithFields(logrus.Fields{"tag": "stderr", "app_id": call.AppID, "fn_id": call.FnID, "image": call.Image, "container_id": id}),
|
||||
}}
|
||||
|
||||
stdout.Swap(newLineWriterWithBuffer(buf1, soc))
|
||||
stderr.Swap(newLineWriterWithBuffer(buf2, sec))
|
||||
}
|
||||
stdout.Swap(newLineWriterWithBuffer(buf1, soc))
|
||||
stderr.Swap(newLineWriterWithBuffer(buf2, sec))
|
||||
|
||||
var iofs iofs
|
||||
var err error
|
||||
if call.Format == models.FormatHTTPStream {
|
||||
// XXX(reed): we should also point stdout to stderr, and not have stdin
|
||||
if cfg.IOFSEnableTmpfs {
|
||||
iofs, err = newTmpfsIOFS(ctx, cfg)
|
||||
} else {
|
||||
iofs, err = newDirectoryIOFS(ctx, cfg)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if cfg.IOFSEnableTmpfs {
|
||||
iofs, err = newTmpfsIOFS(ctx, cfg)
|
||||
} else {
|
||||
iofs = &noopIOFS{}
|
||||
iofs, err = newDirectoryIOFS(ctx, cfg)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &container{
|
||||
@@ -1359,11 +1131,9 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
||||
{Name: "fn_id", Value: call.FnID},
|
||||
},
|
||||
},
|
||||
stdin: stdin,
|
||||
stdout: stdout,
|
||||
stderr: stderr,
|
||||
close: func() {
|
||||
stdin.Close()
|
||||
stderr.Close()
|
||||
stdout.Close()
|
||||
for _, b := range bufs {
|
||||
@@ -1378,9 +1148,8 @@ func newHotContainer(ctx context.Context, call *call, cfg *Config) (*container,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *container) swap(stdin io.Reader, stdout, stderr io.Writer, cs *drivers.Stats) func() {
|
||||
func (c *container) swap(stdout, stderr io.Writer, cs *drivers.Stats) func() {
|
||||
// if tests don't catch this, then fuck me
|
||||
ostdin := c.stdin.(common.GhostReader).Swap(stdin)
|
||||
ostdout := c.stdout.(common.GhostWriter).Swap(stdout)
|
||||
ostderr := c.stderr.(common.GhostWriter).Swap(stderr)
|
||||
|
||||
@@ -1390,7 +1159,6 @@ func (c *container) swap(stdin io.Reader, stdout, stderr io.Writer, cs *drivers.
|
||||
c.swapMu.Unlock()
|
||||
|
||||
return func() {
|
||||
c.stdin.(common.GhostReader).Swap(ostdin)
|
||||
c.stdout.(common.GhostWriter).Swap(ostdout)
|
||||
c.stderr.(common.GhostWriter).Swap(ostderr)
|
||||
c.swapMu.Lock()
|
||||
@@ -1401,13 +1169,13 @@ func (c *container) swap(stdin io.Reader, stdout, stderr io.Writer, cs *drivers.
|
||||
|
||||
func (c *container) Id() string { return c.id }
|
||||
func (c *container) Command() string { return "" }
|
||||
func (c *container) Input() io.Reader { return c.stdin }
|
||||
func (c *container) Input() io.Reader { return nil }
|
||||
func (c *container) Logger() (io.Writer, io.Writer) { return c.stdout, c.stderr }
|
||||
func (c *container) Volumes() [][2]string { return nil }
|
||||
func (c *container) WorkDir() string { return "" }
|
||||
func (c *container) Close() { c.close() }
|
||||
func (c *container) Image() string { return c.image }
|
||||
func (c *container) Timeout() time.Duration { return c.timeout }
|
||||
func (c *container) Timeout() time.Duration { return 0 } // context handles this
|
||||
func (c *container) EnvVars() map[string]string { return c.env }
|
||||
func (c *container) Memory() uint64 { return c.memory * 1024 * 1024 } // convert MB
|
||||
func (c *container) CPUs() uint64 { return c.cpus }
|
||||
|
||||
Reference in New Issue
Block a user